You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/28 16:45:16 UTC

[GitHub] [flink] XComp opened a new pull request #14798: [FLINK-21187] Provide exception history for root causes

XComp opened a new pull request #14798:
URL: https://github.com/apache/flink/pull/14798


   ## What is the purpose of the change
   
   This PR handles the initial exception history proposal collecting only the root causes. This is part of a major effort of providing a exception history to the Flink web UI ([FLINK-6042](https://issues.apache.org/jira/browse/FLINK-6042)).
   
   
   ## Brief change log
   
   * updated `AccessExecutionGraph` interface to return `ErrorInfo` instead of a stringified version of the failure.
   * FailureHandlingResult get a new member identifying the `ExecutionVertex` that failed.
   * Added two maps collecting the error infos for each occurring failure root cause.
   
   ## Verifying this change
   
   * `ExecutionFailureHandlerTest` and `FailureHandlingResultTest` were updated accordingly.
   * `DefaultSchedulerTest#testExceptionHistoryWithRestartableFailure` was introduced to cover the exception history collection.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r566976287



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -164,6 +168,8 @@
     private final Map<OperatorID, OperatorCoordinatorHolder> coordinatorMap;
 
     private final ComponentMainThreadExecutor mainThreadExecutor;
+    private final Map<ExecutionAttemptID, ErrorInfo> localTaskFailures = new HashMap<>();
+    private final Map<Set<ExecutionAttemptID>, ErrorInfo> globalTaskFailures = new HashMap<>();

Review comment:
       For what do we need the maps here? Wouldn't a single `List<ErrorInfo>` be good enough here? Is this kind of a preparation for supporting to capture the exceptions of multiple `Executions`?




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * b895dddd2a1833c10dd04f2823dd19e497fc882c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337) 
   * 8e732bfb2bddc38ec7422f482dcda4be3d296408 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951) 
   * 5910b1a876d28886a8b5f87c09e67d75d2a45cd3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a9204577129b7145846210351ee8754928287a3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r568774178



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -100,17 +126,26 @@ public long getRestartDelayMS() {
         }
     }
 
+    /**
+     * Returns an {@code Optional} with the {@link ExecutionVertexID} of the task causing this
+     * failure or an empty {@code Optional} if it's a global failure.
+     *
+     * @return The {@code ExecutionVertexID} of the causing task or an empty {@code Optional} if
+     *     it's a global failure.
+     */
+    public Optional<ExecutionVertexID> getExecutionVertexIdOfFailedTask() {
+        return failingExecutionVertexId == null
+                ? Optional.empty()
+                : Optional.of(failingExecutionVertexId);

Review comment:
       ```suggestion
           return Optional.ofNullable(failingExecutionVertexId);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecution.java
##########
@@ -121,8 +125,8 @@ public AllocationID getAssignedAllocationID() {
     }
 
     @Override
-    public String getFailureCauseAsString() {
-        return failureCause;
+    public Optional<ErrorInfo> getFailureInfo() {
+        return failureInfo == null ? Optional.empty() : Optional.of(failureInfo);

Review comment:
       ```suggestion
           return Optional.ofNullable(failureInfo);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -522,6 +527,7 @@ protected ComponentMainThreadExecutor getMainThreadExecutor() {
     protected void failJob(Throwable cause) {
         incrementVersionsOfAllVertices();
         executionGraph.failJob(cause);
+        getTerminationFuture().thenRun(() -> archiveGlobalFailure(cause));

Review comment:
       I think we should check here that we are indeed in the `FAILED` state. What can also happen is that the user cancels the job in between.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -317,13 +316,11 @@ public TaskManagerLocation getAssignedResourceLocation() {
                 : null;
     }
 
-    public Throwable getFailureCause() {
-        return failureCause;
-    }
-
     @Override
-    public String getFailureCauseAsString() {
-        return ExceptionUtils.stringifyException(getFailureCause());
+    public Optional<ErrorInfo> getFailureInfo() {
+        return failureCause == null
+                ? Optional.empty()
+                : Optional.of(new ErrorInfo(failureCause, getStateTimestamp(FAILED)));

Review comment:
       Does it make sense to change `failureCause` to be of type `ErrorInfo`? If yes, then we could save recreating this object multiple times.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
##########
@@ -371,9 +375,12 @@ private static void compareExecution(
         assertEquals(
                 runtimeExecution.getAssignedResourceLocation(),
                 archivedExecution.getAssignedResourceLocation());
-        assertEquals(
-                runtimeExecution.getFailureCauseAsString(),
-                archivedExecution.getFailureCauseAsString());
+        assertThat(
+                runtimeExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString),
+                is(archivedExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString)));
+        assertThat(
+                runtimeExecution.getFailureInfo().map(ErrorInfo::getTimestamp),
+                is(archivedExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString)));

Review comment:
       Is this correct? Why does CI passes with this assertion if it is wrong?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));
+        assertThat(
+                globalFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.lowerEndpoint()));
+        assertThat(
+                globalFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.upperEndpoint()));
+    }
+
+    private static Range<Long> initiateFailure(
+            DefaultScheduler scheduler,
+            JobID jobId,
+            ExecutionAttemptID executionAttemptID,
+            Throwable exception) {
+        long start = System.currentTimeMillis();

Review comment:
       Yes, we are actually interested in a timestamp and not some measure of passed time. But for testing purposes a manually controlled `Clock` can help to avoid instabilities cause by `System.currentTimeMillis`. The production code could then still use `System.currentTimeMillis`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecution.java
##########
@@ -64,9 +66,11 @@
      * Returns the exception that caused the job to fail. This is the first root exception that was
      * not recoverable and triggered job failure.
      *
-     * @return failure exception as a string, or {@code "(null)"}
+     * @return an {@code Optional} of {@link ErrorInfo} containing the {@code Throwable} wrapped in
+     *     a {@link SerializedThrowable} and the time it was registered if an error occurred. If no

Review comment:
       nit: That the throwable is wrapped in a `SerializedThrowable` should be an implementation detail of the `ErrorInfo`. I think it is not necessary to mention it here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -641,6 +647,32 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(new ErrorInfo(failure, System.currentTimeMillis()));

Review comment:
       I would suggest to take the `ExecutionGraph.getStatusTimestamp`. That way the timestamp will be consistent.




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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-779102405


   I looked through the code (supported by @AHeise): The race condition really only kicks in when cancelling/failing the task because that's when the `failureCause` becomes relevant. So, instead of synchronizing the [state transition](https://github.com/XComp/flink/blob/5781449f38c1e36c1a2952518f9e30761d915f04/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L1052) we could add synchronize blocks for the [cancellation of a task](https://github.com/XComp/flink/blob/5781449f38c1e36c1a2952518f9e30761d915f04/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L1130) and while [handling failure handling during the normal Invokable execution](https://github.com/XComp/flink/blob/5781449f38c1e36c1a2952518f9e30761d915f04/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L828). The synchronization will only cover the state transition and setting the `failureCause`. Cancelling the corresponding task would be moved out of 
 the synchronization block.


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579197850



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -100,17 +131,24 @@ public long getRestartDelayMS() {
         }
     }
 
+    /**
+     * Returns an {@code Optional} with the {@link ExecutionVertexID} of the task causing this
+     * failure or an empty {@code Optional} if it's a global failure.
+     *
+     * @return The {@code ExecutionVertexID} of the causing task or an empty {@code Optional} if
+     *     it's a global failure.
+     */
+    public Optional<ExecutionVertexID> getExecutionVertexIdOfFailedTask() {
+        return Optional.ofNullable(failingExecutionVertexId);
+    }
+
     /**
      * Returns reason why the restarting cannot be conducted.
      *
      * @return reason why the restarting cannot be conducted
      */
     public Throwable getError() {

Review comment:
       `@Nullable` is missing. One way to find nullability problems is to turn on IntelliJ's inspections. There are couple of those for null checks. This helps me quite a lot.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,41 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        archiveGlobalFailure(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED));
+    }
+
+    protected void archiveGlobalFailure(Throwable failure, long timestamp) {
+        taskFailureHistory.add(new ErrorInfo(failure, timestamp));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {

Review comment:
       Shall we make these methods `final` since we don't want them to be overriden.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,41 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        archiveGlobalFailure(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED));
+    }
+
+    protected void archiveGlobalFailure(Throwable failure, long timestamp) {
+        taskFailureHistory.add(new ErrorInfo(failure, timestamp));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        final Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        if (executionOptional.isPresent()) {
+            final Execution failedExecution = executionOptional.get();
+            failedExecution
+                    .getFailureInfo()
+                    .ifPresent(
+                            failureInfo -> {
+                                taskFailureHistory.add(failureInfo);
+                                log.debug(
+                                        "Archive local failure causing attempt {} to fail: {}",
+                                        failedExecution.getAttemptId(),
+                                        failureInfo.getExceptionAsString());
+                            });
+        } else {
+            // fallback in case of a global fail over - no failed state is set and, therefore, no
+            // timestamp was taken
+            archiveGlobalFailure(failureHandlingResult.getError(), System.currentTimeMillis());

Review comment:
       I think `getError` can return `null`. This needs to be handled properly.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,41 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        archiveGlobalFailure(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED));
+    }
+
+    protected void archiveGlobalFailure(Throwable failure, long timestamp) {
+        taskFailureHistory.add(new ErrorInfo(failure, timestamp));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        final Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        if (executionOptional.isPresent()) {

Review comment:
       Maybe it is a bit clearer to use `failureHandlingResult.isGlobalFailure()` because at the moment the contract is that `executionVertexIdOfFailedTask == null` iff it's a global failure.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 94d299053103c1bec5dfefaf4089ed0b31c403bb Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321) 
   * b895dddd2a1833c10dd04f2823dd19e497fc882c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 854b645872b5596d2cc618c65dac0228c616eec2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-775012926


   Ok, then let's first fix this instability before merging this PR @XComp. Ping once the instability has been fixed.


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r578737299



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       Because of the timestamp in case of local failure: The `Execution` fails but does not necessarily trigger the `ExecutionGraph` to switch to fail. Hence, no timestamp is record for this kind of failure.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r569720896



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
##########
@@ -371,9 +375,12 @@ private static void compareExecution(
         assertEquals(
                 runtimeExecution.getAssignedResourceLocation(),
                 archivedExecution.getAssignedResourceLocation());
-        assertEquals(
-                runtimeExecution.getFailureCauseAsString(),
-                archivedExecution.getFailureCauseAsString());
+        assertThat(
+                runtimeExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString),
+                is(archivedExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString)));
+        assertThat(
+                runtimeExecution.getFailureInfo().map(ErrorInfo::getTimestamp),
+                is(archivedExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString)));

Review comment:
       Oh man, good catch. `ArchiveExecutionGraphTest` triggered a global failure which only set the `failureInfo` on the `ExecutionGraph` level. I switched to `SchedulerBase.updateTaskExecutionState` which also sets the `failureInfo` on an `Execution` level




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cde4fbdc8d036cdb44884a0974fbf1cf31966c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912) 
   * 211a68e9532b0bf10581388b395e474123ec6ad5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e962af6d8df77a725defeb92537837e6f685292f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487",
       "triggerID" : "e962af6d8df77a725defeb92537837e6f685292f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * e962af6d8df77a725defeb92537837e6f685292f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911






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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r570084862



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -522,6 +527,7 @@ protected ComponentMainThreadExecutor getMainThreadExecutor() {
     protected void failJob(Throwable cause) {
         incrementVersionsOfAllVertices();
         executionGraph.failJob(cause);
+        getTerminationFuture().thenRun(() -> archiveGlobalFailure(cause));

Review comment:
       I mean the case that a failure happens, the job goes into the `FAILING` state and tries to cancel the tasks and now the user cancels the job because it takes too long for him. Then the job will go into the `CANCELING` state which will result to the `CANCELED` state once all tasks have terminated.
   
   I think you are right that we should still record the failure 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:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r578786368



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       But looking into that again, I realize that there is, indeed, the `failureInfo` in `ExecutionGraph` that is set during [handleTaskFailure](https://github.com/XComp/flink/blob/8e732bfb2bddc38ec7422f482dcda4be3d296408/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java#L218) and [handleGlobalFailure](https://github.com/XComp/flink/blob/8e732bfb2bddc38ec7422f482dcda4be3d296408/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java#L236). This is used to forward the current root cause of the `ExecutionGraph` to the Web UI. I could have used that. But the drawback is that it's using `System.currentMillis`.
   
   Instead, I'm gonna work on removing the `ExecutionGraph.failureInfo` (and related methods) as part of FLINK-21188. It becomes redundant after we enable to exception history since it's just the exception history's last entry.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8f6541c7820d1eea9c29c915690070cfebe837b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a9204577129b7145846210351ee8754928287a3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620) 
   * 22d7f6d27419e0f58b738b10e3a05a64ef2f24dd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r567417737



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -164,6 +168,8 @@
     private final Map<OperatorID, OperatorCoordinatorHolder> coordinatorMap;
 
     private final ComponentMainThreadExecutor mainThreadExecutor;
+    private final Map<ExecutionAttemptID, ErrorInfo> localTaskFailures = new HashMap<>();
+    private final Map<Set<ExecutionAttemptID>, ErrorInfo> globalTaskFailures = new HashMap<>();

Review comment:
       You're right. That was still some leftover of my initial work considering multiple exceptions per failure cause. I'm gonna refactor it to use a List here in order to be more consistent.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 22d7f6d27419e0f58b738b10e3a05a64ef2f24dd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e962af6d8df77a725defeb92537837e6f685292f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e962af6d8df77a725defeb92537837e6f685292f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 854b645872b5596d2cc618c65dac0228c616eec2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478) 
   * e962af6d8df77a725defeb92537837e6f685292f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r578786368



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       But looking into that again, I realize that there is, indeed, the `failureInfo` in `ExecutionGraph` that is set during [handleTaskFailure](https://github.com/XComp/flink/blob/8e732bfb2bddc38ec7422f482dcda4be3d296408/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java#L218) and [handleGlobalFailure](https://github.com/XComp/flink/blob/8e732bfb2bddc38ec7422f482dcda4be3d296408/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java#L236). This is used to forward the current root cause of the `ExecutionGraph` to the Web UI. I could have used that. But the drawback is that it's using `System.currentMillis`.
   
   Instead, I'm gonna work on removing the `ExecutionGraph.failureInfo` (and related methods) as part of FLINK-21190. It becomes redundant after we enable to exception history since it's just the exception history's last entry.




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r578407995



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -153,7 +152,7 @@
 
     private LogicalSlot assignedResource;
 
-    private Throwable failureCause; // once assigned, never changes
+    private Optional<ErrorInfo> failureCause = Optional.empty(); // once assigned, never changes

Review comment:
       The comment seems to be wrong now.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -1133,7 +1128,10 @@ private void processFail(
         checkState(transitionState(current, FAILED, t));
 
         // success (in a manner of speaking)
-        this.failureCause = t;
+        if (t != null) {
+            // we only set the failureCause if an error is passed (see FLINK-21376)
+            this.failureCause = Optional.of(new ErrorInfo(t, getStateTimestamp(FAILED)));
+        }

Review comment:
       Can't we say that if `t == null`, then we create an unknown failure cause? `new FlinkException("Unknown cause for Execution failure. This might be caused by FLINK-21376")`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);

Review comment:
       I think `archiveFromFailureHandlingResult` can also be called when handling a global failover. In this case `executionOptional` would be empty and as a consequence we won't record a failure cause.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       Why do we have to ask the execution for the failure cause? If we are only interested in the root cause, doesn't `failureHandlingResult` contain all the required information?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -38,6 +42,12 @@
     /** Delay before the restarting can be conducted. */
     private final long restartDelayMS;
 
+    /**
+     * The ExecutionVertexID refering to the ExecutionVertex the failure is originating from or
+     * {@code null} if it's a global failure.
+     */
+    @Nullable private final ExecutionVertexID failingExecutionVertexId;
+
     /** Reason why the failure is not recoverable. */

Review comment:
       The JavaDoc seems wrong. If I am not mistaken, then we will also set this field if the failure is recoverable, right?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +871,87 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {

Review comment:
       I think we also need a test for global failover.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951) 
   * 5910b1a876d28886a8b5f87c09e67d75d2a45cd3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769218666


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit e962af6d8df77a725defeb92537837e6f685292f (Fri Feb 19 07:34:36 UTC 2021)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5910b1a876d28886a8b5f87c09e67d75d2a45cd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769218666


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 9a9204577129b7145846210351ee8754928287a3 (Thu Jan 28 16:47:59 UTC 2021)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-21187).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 94d299053103c1bec5dfefaf4089ed0b31c403bb Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321) 
   * b895dddd2a1833c10dd04f2823dd19e497fc882c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 94d299053103c1bec5dfefaf4089ed0b31c403bb Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285) 
   * 94d299053103c1bec5dfefaf4089ed0b31c403bb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5910b1a876d28886a8b5f87c09e67d75d2a45cd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962) 
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r568774178



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -100,17 +126,26 @@ public long getRestartDelayMS() {
         }
     }
 
+    /**
+     * Returns an {@code Optional} with the {@link ExecutionVertexID} of the task causing this
+     * failure or an empty {@code Optional} if it's a global failure.
+     *
+     * @return The {@code ExecutionVertexID} of the causing task or an empty {@code Optional} if
+     *     it's a global failure.
+     */
+    public Optional<ExecutionVertexID> getExecutionVertexIdOfFailedTask() {
+        return failingExecutionVertexId == null
+                ? Optional.empty()
+                : Optional.of(failingExecutionVertexId);

Review comment:
       ```suggestion
           return Optional.ofNullable(failingExecutionVertexId);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecution.java
##########
@@ -121,8 +125,8 @@ public AllocationID getAssignedAllocationID() {
     }
 
     @Override
-    public String getFailureCauseAsString() {
-        return failureCause;
+    public Optional<ErrorInfo> getFailureInfo() {
+        return failureInfo == null ? Optional.empty() : Optional.of(failureInfo);

Review comment:
       ```suggestion
           return Optional.ofNullable(failureInfo);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -522,6 +527,7 @@ protected ComponentMainThreadExecutor getMainThreadExecutor() {
     protected void failJob(Throwable cause) {
         incrementVersionsOfAllVertices();
         executionGraph.failJob(cause);
+        getTerminationFuture().thenRun(() -> archiveGlobalFailure(cause));

Review comment:
       I think we should check here that we are indeed in the `FAILED` state. What can also happen is that the user cancels the job in between.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -317,13 +316,11 @@ public TaskManagerLocation getAssignedResourceLocation() {
                 : null;
     }
 
-    public Throwable getFailureCause() {
-        return failureCause;
-    }
-
     @Override
-    public String getFailureCauseAsString() {
-        return ExceptionUtils.stringifyException(getFailureCause());
+    public Optional<ErrorInfo> getFailureInfo() {
+        return failureCause == null
+                ? Optional.empty()
+                : Optional.of(new ErrorInfo(failureCause, getStateTimestamp(FAILED)));

Review comment:
       Does it make sense to change `failureCause` to be of type `ErrorInfo`? If yes, then we could save recreating this object multiple times.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
##########
@@ -371,9 +375,12 @@ private static void compareExecution(
         assertEquals(
                 runtimeExecution.getAssignedResourceLocation(),
                 archivedExecution.getAssignedResourceLocation());
-        assertEquals(
-                runtimeExecution.getFailureCauseAsString(),
-                archivedExecution.getFailureCauseAsString());
+        assertThat(
+                runtimeExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString),
+                is(archivedExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString)));
+        assertThat(
+                runtimeExecution.getFailureInfo().map(ErrorInfo::getTimestamp),
+                is(archivedExecution.getFailureInfo().map(ErrorInfo::getExceptionAsString)));

Review comment:
       Is this correct? Why does CI passes with this assertion if it is wrong?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));
+        assertThat(
+                globalFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.lowerEndpoint()));
+        assertThat(
+                globalFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.upperEndpoint()));
+    }
+
+    private static Range<Long> initiateFailure(
+            DefaultScheduler scheduler,
+            JobID jobId,
+            ExecutionAttemptID executionAttemptID,
+            Throwable exception) {
+        long start = System.currentTimeMillis();

Review comment:
       Yes, we are actually interested in a timestamp and not some measure of passed time. But for testing purposes a manually controlled `Clock` can help to avoid instabilities cause by `System.currentTimeMillis`. The production code could then still use `System.currentTimeMillis`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecution.java
##########
@@ -64,9 +66,11 @@
      * Returns the exception that caused the job to fail. This is the first root exception that was
      * not recoverable and triggered job failure.
      *
-     * @return failure exception as a string, or {@code "(null)"}
+     * @return an {@code Optional} of {@link ErrorInfo} containing the {@code Throwable} wrapped in
+     *     a {@link SerializedThrowable} and the time it was registered if an error occurred. If no

Review comment:
       nit: That the throwable is wrapped in a `SerializedThrowable` should be an implementation detail of the `ErrorInfo`. I think it is not necessary to mention it here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -641,6 +647,32 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(new ErrorInfo(failure, System.currentTimeMillis()));

Review comment:
       I would suggest to take the `ExecutionGraph.getStatusTimestamp`. That way the timestamp will be consistent.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cde4fbdc8d036cdb44884a0974fbf1cf31966c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 95f80b654417779460301440b21d1c4d1b9476e3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8f6541c7820d1eea9c29c915690070cfebe837b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644) 
   * f6278de3bae76474c390b6251551e68862e3841f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 95f80b654417779460301440b21d1c4d1b9476e3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276) 
   * 06115e6dc53dfd654db8ee33449352d8f43abd89 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579261225



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,41 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        archiveGlobalFailure(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED));
+    }
+
+    protected void archiveGlobalFailure(Throwable failure, long timestamp) {
+        taskFailureHistory.add(new ErrorInfo(failure, timestamp));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        final Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        if (executionOptional.isPresent()) {

Review comment:
       Fair enough. Then let's not change it.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 22d7f6d27419e0f58b738b10e3a05a64ef2f24dd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621) 
   * b8f6541c7820d1eea9c29c915690070cfebe837b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r567422474



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));
+        assertThat(
+                globalFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.lowerEndpoint()));
+        assertThat(
+                globalFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.upperEndpoint()));
+    }
+
+    private static Range<Long> initiateFailure(
+            DefaultScheduler scheduler,
+            JobID jobId,
+            ExecutionAttemptID executionAttemptID,
+            Throwable exception) {
+        long start = System.currentTimeMillis();

Review comment:
       Correct me if I'm wrong here: Another way to measure the time would be `System.nanoTime()` since it ensures monotonicity. But using this in the test would require `Execution` to use `System.nanoTime()` as well which would not work as `System.nanoTime` cannot be used to compare timestamps between different JVMs [[1]](https://www.javaadvent.com/2019/12/measuring-time-from-java-to-kernel-and-back.html)?!




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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579234673



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,41 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        archiveGlobalFailure(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED));
+    }
+
+    protected void archiveGlobalFailure(Throwable failure, long timestamp) {
+        taskFailureHistory.add(new ErrorInfo(failure, timestamp));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        final Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        if (executionOptional.isPresent()) {

Review comment:
       I'm hesitant to change that: IMHO, it makes the code harder to read and easier to misbehave. The current version of the code is straight forward: Use the `FAILED` timestamp provided the corresponding `Execution` if we have the corresponding `ExecutionAttemptId`. Additionally, your change request might cause unwanted behavior if we decide to introduce global failures due to some local failure. I consider the fact that the global failure does not have a causing `Execution` an implementation detail which we don't have to expose here. @tillrohrmann Is that reasonable?




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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579259796



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       Yes, that would be possible if would would like to relax the constraint that the exception history does not provide the timestamp the `Execution` was marked as failed but an earlier timestamp. But to be fair, we do that already for the fallback of a global failover in `archiveFromFailureHandlingResult`.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6278de3bae76474c390b6251551e68862e3841f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710) 
   * 0cde4fbdc8d036cdb44884a0974fbf1cf31966c5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r567418118



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));

Review comment:
       Ah, thanks for the hint. I didn't know about `FlinkMatchers`, yet.




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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-773838859


   The test failures reported by Azure seem to be related to [FLINK-21277](https://issues.apache.org/jira/browse/FLINK-21277). I'm going to rebase and squash the related commits together after @rkhachatryan gave his final ok


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



[GitHub] [flink] XComp edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-774663350


   There seems to be some test instability in the fine-grained source data now with the invariant being added. The [Azure build attached to this PR](https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=13005&view=logs&j=a57e0635-3fad-5b08-57c7-a4142d7d6fa9&t=5360d54c-8d94-5d85-304e-a89267eb785a) succeeded. [Another Azure build on the same commit](https://dev.azure.com/mapohl/flink/_build/results?buildId=221&view=logs&j=cc649950-03e9-5fae-8326-2f1ad744b536&t=51cab6ca-669f-5dc0-221d-1e4f7dc4fc85&l=5837) failed in `TaskExecutorSlotLifetimeTest`. The test failure is caused by the newly introduced Precondition. To me, that looks like an issue in the fine-grained resource management code or some other parts of the code which we should investigate. I'd prefer to create a Jira issue covering this instead of relaxing the state change conditions again and work around it in the exception history. I still think that the invariant makes sense and we shouldn't relax it.


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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-782147661


   I addressed all changes, squashed and rebased the branch.


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579274752



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       I added this remark to FLINK-21190




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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-773469973






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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-773469973


   @pnowojski @AHeise @rkhachatryan Could one of you guys have a look at the `Task`-related change I did in [211a68e](https://github.com/apache/flink/pull/14798/commits/211a68e9532b0bf10581388b395e474123ec6ad5)?


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8f6541c7820d1eea9c29c915690070cfebe837b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644) 
   * f6278de3bae76474c390b6251551e68862e3841f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 8e732bfb2bddc38ec7422f482dcda4be3d296408 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413) 
   * 854b645872b5596d2cc618c65dac0228c616eec2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r578787023



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);

Review comment:
       You're right. I remember looking into that. Back then, I mistook the task as only focusing on "task failures". That was a misunderstanding. I added the test for global failover and the test.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 8e732bfb2bddc38ec7422f482dcda4be3d296408 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413) 
   * 854b645872b5596d2cc618c65dac0228c616eec2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * b895dddd2a1833c10dd04f2823dd19e497fc882c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337) 
   * 8e732bfb2bddc38ec7422f482dcda4be3d296408 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-779221947


   I created [FLINK-21376](https://issues.apache.org/jira/browse/FLINK-21376) to cover the `failureCause`/`executionState` synchronization. The code changes of FLINK-21187 will also handle cases where no `failureCause` is set for now.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285) 
   * 94d299053103c1bec5dfefaf4089ed0b31c403bb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r570012364



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));
+        assertThat(
+                globalFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.lowerEndpoint()));
+        assertThat(
+                globalFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.upperEndpoint()));
+    }
+
+    private static Range<Long> initiateFailure(
+            DefaultScheduler scheduler,
+            JobID jobId,
+            ExecutionAttemptID executionAttemptID,
+            Throwable exception) {
+        long start = System.currentTimeMillis();

Review comment:
       Thanks for clarification.




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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-774075110


   @tillrohrmann I addressed all comments, squashed the changes and rebased the branch.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 95f80b654417779460301440b21d1c4d1b9476e3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276) 
   * 06115e6dc53dfd654db8ee33449352d8f43abd89 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284) 
   * 354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r566868391



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionVertex.java
##########
@@ -64,9 +64,10 @@
      * Returns the exception that caused the job to fail. This is the first root exception that was
      * not recoverable and triggered job failure.
      *
-     * @return failure exception as a string, or {@code "(null)"}
+     * @return failure exception wrapped in a {@link ErrorInfo}, or {@code null} if no exception was
+     *     caught.
      */
-    String getFailureCauseAsString();
+    ErrorInfo getFailureInfo();

Review comment:
       Same here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecution.java
##########
@@ -64,9 +65,11 @@
      * Returns the exception that caused the job to fail. This is the first root exception that was
      * not recoverable and triggered job failure.
      *
-     * @return failure exception as a string, or {@code "(null)"}
+     * @return {@link ErrorInfo} containing the {@code Throwable} wrapped in a {@link
+     *     SerializedThrowable} and the time it was registered, or {@code null} if no exception was
+     *     caught.
      */
-    String getFailureCauseAsString();
+    ErrorInfo getFailureInfo();

Review comment:
       If this method returns `null`, then I would suggest to return an `Optional<ErrorInfo>`. This makes the contract clearer.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecution.java
##########
@@ -70,14 +69,14 @@ public ArchivedExecution(
             ExecutionAttemptID attemptId,
             int attemptNumber,
             ExecutionState state,
-            String failureCause,
+            ErrorInfo failureCause,

Review comment:
       `@Nullable` missing.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecution.java
##########
@@ -37,7 +36,7 @@
 
     private final ExecutionState state;
 
-    private final String failureCause; // once assigned, never changes
+    private final ErrorInfo failureInfo; // once assigned, never changes

Review comment:
       I guess that this field is `@Nullable`, right?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java
##########
@@ -99,23 +103,31 @@ public FailureHandlingResult getGlobalFailureHandlingResult(final Throwable caus
     }
 
     private FailureHandlingResult handleFailure(
+            final ExecutionVertexID failingExecutionVertexId,

Review comment:
       I think `@Nullable` annotation is missing here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -322,8 +322,11 @@ public Throwable getFailureCause() {
     }
 
     @Override
-    public String getFailureCauseAsString() {
-        return ExceptionUtils.stringifyException(getFailureCause());
+    public ErrorInfo getFailureInfo() {
+        return getFailureCause() == null
+                ? null
+                : new ErrorInfo(
+                        new SerializedThrowable(getFailureCause()), getStateTimestamp(FAILED));

Review comment:
       I think wrapping `getFailureCause` in a `SerializedThrowable` is not necessary because that's what `ErrorInfo` does in the constructor.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
##########
@@ -351,7 +351,19 @@ private static void compareExecutionVertex(
                 runtimeVertex.getStateTimestamp(ExecutionState.FAILED),
                 archivedVertex.getStateTimestamp(ExecutionState.FAILED));
         assertEquals(
-                runtimeVertex.getFailureCauseAsString(), archivedVertex.getFailureCauseAsString());
+                runtimeVertex.getFailureInfo() == null
+                        ? null
+                        : runtimeVertex.getFailureInfo().getExceptionAsString(),
+                archivedVertex.getFailureInfo() == null
+                        ? null
+                        : archivedVertex.getFailureInfo().getExceptionAsString());

Review comment:
       with `Optional<ErrorInfo>` this might look a bit nicer.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -100,17 +116,22 @@ public long getRestartDelayMS() {
         }
     }
 
+    /**
+     * Returns the {@link ExecutionVertexID} of the task causing this failure.
+     *
+     * @return The {@code ExecutionVertexID} or {@code null} if it's a global failure.
+     */
+    public ExecutionVertexID getExecutionVertexIdOfFailedTask() {

Review comment:
       `@Nullable` is missing.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -38,6 +39,9 @@
     /** Delay before the restarting can be conducted. */
     private final long restartDelayMS;
 
+    /** The ExecutionVertexID refering to the ExecutionVertex the failure is originating from. */
+    private final ExecutionVertexID failingExecutionVertexId;

Review comment:
       I think `@Nullable` is missing here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -47,27 +51,39 @@
     /**
      * Creates a result of a set of tasks to restart to recover from the failure.
      *
+     * @param failingExecutionVertexId the {@link ExecutionVertexID} refering to the {@link
+     *     ExecutionVertex} the failure is originating from.

Review comment:
       Explanation missing what `null` value means.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -267,9 +267,16 @@ private void restartTasksWithDelay(final FailureHandlingResult failureHandlingRe
         delayExecutor.schedule(
                 () ->
                         FutureUtils.assertNoException(
-                                cancelFuture.thenRunAsync(
-                                        restartTasks(executionVertexVersions, globalRecovery),
-                                        getMainThreadExecutor())),
+                                cancelFuture
+                                        .thenRunAsync(
+                                                () ->
+                                                        archiveFromFailureHandlingResult(
+                                                                failureHandlingResult),
+                                                getMainThreadExecutor())
+                                        .thenRunAsync(
+                                                restartTasks(
+                                                        executionVertexVersions, globalRecovery),
+                                                getMainThreadExecutor())),

Review comment:
       Why did you introduce a second `thenRunAsync`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));

Review comment:
       One could use `FlinkMatchers.containsCause`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -641,6 +648,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        Set<ExecutionAttemptID> executionAttemptIds =
+                IterableUtils.toStream(executionGraph.getAllExecutionVertices())
+                        .map(ExecutionVertex::getCurrentExecutionAttempt)
+                        .map(Execution::getAttemptId)
+                        .collect(Collectors.toSet());
+
+        globalTaskFailures.put(
+                executionAttemptIds, new ErrorInfo(failure, System.currentTimeMillis()));
+
+        log.debug("Archive global {}: {}", failure, failure.getMessage());

Review comment:
       How would this log line look like? Wouldn't `failure.getMessage` be duplicated? Maybe also say "Archive global failure...".

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -641,6 +648,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        Set<ExecutionAttemptID> executionAttemptIds =
+                IterableUtils.toStream(executionGraph.getAllExecutionVertices())
+                        .map(ExecutionVertex::getCurrentExecutionAttempt)
+                        .map(Execution::getAttemptId)
+                        .collect(Collectors.toSet());
+
+        globalTaskFailures.put(
+                executionAttemptIds, new ErrorInfo(failure, System.currentTimeMillis()));
+
+        log.debug("Archive global {}: {}", failure, failure.getMessage());
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        if (failureHandlingResult.getExecutionVertexIdOfFailedTask() == null) {

Review comment:
       I think it is nicer if you have something like `failureHandlingResult.isGlobalFailure()`. `failureHandlingResult.getExecutionVertexIdOfFailedTask() == null` is not very expressive.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -100,17 +116,22 @@ public long getRestartDelayMS() {
         }
     }
 
+    /**
+     * Returns the {@link ExecutionVertexID} of the task causing this failure.
+     *
+     * @return The {@code ExecutionVertexID} or {@code null} if it's a global failure.
+     */
+    public ExecutionVertexID getExecutionVertexIdOfFailedTask() {

Review comment:
       Alternatively you can introduce a `isLocalFailure()` method and let this method fail if `failingExecutionVertexId == null`. That way you enforce the contract that you can only ask for the failing execution vertex id if it is a local failure.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -47,27 +51,39 @@
     /**
      * Creates a result of a set of tasks to restart to recover from the failure.
      *
+     * @param failingExecutionVertexId the {@link ExecutionVertexID} refering to the {@link
+     *     ExecutionVertex} the failure is originating from.
+     * @param cause the exception that caused this failure.
      * @param verticesToRestart containing task vertices to restart to recover from the failure
      * @param restartDelayMS indicate a delay before conducting the restart
      */
     private FailureHandlingResult(
-            Set<ExecutionVertexID> verticesToRestart, long restartDelayMS, boolean globalFailure) {
+            ExecutionVertexID failingExecutionVertexId,

Review comment:
       `@Nullable` is missing.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -164,6 +168,8 @@
     private final Map<OperatorID, OperatorCoordinatorHolder> coordinatorMap;
 
     private final ComponentMainThreadExecutor mainThreadExecutor;
+    private final Map<ExecutionAttemptID, ErrorInfo> localTaskFailures = new HashMap<>();
+    private final Map<Set<ExecutionAttemptID>, ErrorInfo> globalTaskFailures = new HashMap<>();

Review comment:
       For what do we need the maps here? Wouldn't a single `List<ErrorInfo>` be good enough here?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));
+        assertThat(
+                globalFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.lowerEndpoint()));
+        assertThat(
+                globalFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.upperEndpoint()));
+    }
+
+    private static Range<Long> initiateFailure(
+            DefaultScheduler scheduler,
+            JobID jobId,
+            ExecutionAttemptID executionAttemptID,
+            Throwable exception) {
+        long start = System.currentTimeMillis();

Review comment:
       `System.currentTimeMillis()` can be susceptible to clock resets. Hence, it can cause some test instabilities. Unfortunately, `Execution` also uses this to determine the state timestamp.




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579108739



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       So theoretically we would have to add the timestamp to the `FailureHandlingResult` or the `ErrorInfo` if we didn't want to look up the `Execution` again, right?




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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-770378909


   Thanks @tillrohrmann for reviewing the PR. I addressed your changes. I removed code that would become only necessary when adding the support for the exception history collecting concurrent failures to be more consistent.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5910b1a876d28886a8b5f87c09e67d75d2a45cd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962) 
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann closed pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann closed pull request #14798:
URL: https://github.com/apache/flink/pull/14798


   


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r570428422



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
##########
@@ -1022,18 +1021,38 @@ private boolean transitionState(ExecutionState currentState, ExecutionState newS
         return transitionState(currentState, newState, null);
     }
 
+    /**
+     * Try to transition to FAILED state from a given state and sets the {@code failureCause}.
+     *
+     * @param currentState of the execution
+     * @param cause the {@link Throwable} causing the failure
+     * @return true if the transition was successful, otherwise false
+     * @throws NullPointerException if no {@code cause} is provided
+     */
+    private boolean transitionToFailedStateAndSetFailureCause(
+            ExecutionState currentState, Throwable cause) {
+        Preconditions.checkNotNull(cause, "No cause is given when transitioning to FAILED state.");
+        failureCause = cause;
+        return transitionState(currentState, ExecutionState.FAILED, cause);

Review comment:
       I think we can't update `failureCause` until we successfully transitioned to FAILED state.




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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r570520523



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
##########
@@ -1022,18 +1021,38 @@ private boolean transitionState(ExecutionState currentState, ExecutionState newS
         return transitionState(currentState, newState, null);
     }
 
+    /**
+     * Try to transition to FAILED state from a given state and sets the {@code failureCause}.
+     *
+     * @param currentState of the execution
+     * @param cause the {@link Throwable} causing the failure
+     * @return true if the transition was successful, otherwise false
+     * @throws NullPointerException if no {@code cause} is provided
+     */
+    private boolean transitionToFailedStateAndSetFailureCause(
+            ExecutionState currentState, Throwable cause) {
+        Preconditions.checkNotNull(cause, "No cause is given when transitioning to FAILED state.");
+        failureCause = cause;
+        return transitionState(currentState, ExecutionState.FAILED, cause);

Review comment:
       That's actually a good point. I will correct that.




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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-773597349


   @rkhachatryan thanks for looking into it. There was no production code path I ran into that let me start this investigation. I ran into test issues after adding the invariant check to `TaskExecutionState`. That's where I started to check the code to see whether I could rely on the invariant to be true all the time. The only place where the `failureCause` is not set even though it's transitioning into `FAILED` state is [in the fall back case](https://github.com/apache/flink/pull/14798/commits/af0bf68b965c9adaab61f42a99ef3278d27aed32#diff-b84174e55cb1999d99ad60cdeded7be20ff4978472bfc785c5a77b6270f47b56R856).


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 211a68e9532b0bf10581388b395e474123ec6ad5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944) 
   * ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-774663350


   There seems to be some test instability in the fine-grained source data now with the invariant being added. The [Azure build attached to this PR](https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=13005&view=logs&j=a57e0635-3fad-5b08-57c7-a4142d7d6fa9&t=5360d54c-8d94-5d85-304e-a89267eb785a) succeeded. [Another Azure build on the same commit](https://dev.azure.com/mapohl/flink/_build/results?buildId=221&view=logs&j=cc649950-03e9-5fae-8326-2f1ad744b536&t=51cab6ca-669f-5dc0-221d-1e4f7dc4fc85&l=5837) failed in `TaskExecutorSlotLifetimeTest`. The test failure is caused by the newly introduced Precondition. To me, that looks like an issue in the fine-grained resource management code. I'd prefer to create a Jira issue covering this. I still think that the invariant makes sense and we shouldn't relax it.


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



[GitHub] [flink] flinkbot commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a9204577129b7145846210351ee8754928287a3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e962af6d8df77a725defeb92537837e6f685292f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487",
       "triggerID" : "e962af6d8df77a725defeb92537837e6f685292f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f05d87ce3393dca5b8df30919f439b8c272e982e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f05d87ce3393dca5b8df30919f439b8c272e982e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * e962af6d8df77a725defeb92537837e6f685292f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487) 
   * f05d87ce3393dca5b8df30919f439b8c272e982e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r579262135



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -635,6 +641,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        taskFailureHistory.add(
+                new ErrorInfo(failure, executionGraph.getStatusTimestamp(JobStatus.FAILED)));
+        log.debug("Archive global failure.", failure);
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        Optional<Execution> executionOptional =
+                failureHandlingResult
+                        .getExecutionVertexIdOfFailedTask()
+                        .map(this::getExecutionVertex)
+                        .map(ExecutionVertex::getCurrentExecutionAttempt);
+
+        executionOptional.ifPresent(
+                execution ->
+                        execution
+                                .getFailureInfo()
+                                .ifPresent(
+                                        failureInfo -> {
+                                            taskFailureHistory.add(failureInfo);
+                                            log.debug(
+                                                    "Archive local failure causing attempt {} to fail: {}",
+                                                    execution.getAttemptId(),
+                                                    failureInfo.getExceptionAsString());
+                                        }));

Review comment:
       Ok, I think we don't have to change it right now. Maybe we do this sometime in the future.




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



[GitHub] [flink] tillrohrmann commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-774160353


   Thanks for the update. I'll merge this PR once AZP has completed.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e962af6d8df77a725defeb92537837e6f685292f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487",
       "triggerID" : "e962af6d8df77a725defeb92537837e6f685292f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f05d87ce3393dca5b8df30919f439b8c272e982e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13503",
       "triggerID" : "f05d87ce3393dca5b8df30919f439b8c272e982e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * f05d87ce3393dca5b8df30919f439b8c272e982e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13503) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r569986337



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -522,6 +527,7 @@ protected ComponentMainThreadExecutor getMainThreadExecutor() {
     protected void failJob(Throwable cause) {
         incrementVersionsOfAllVertices();
         executionGraph.failJob(cause);
+        getTerminationFuture().thenRun(() -> archiveGlobalFailure(cause));

Review comment:
       Here, I don't understand fully: You're referring to the case where a failure happens, the user cancels the job while the failure handling is done and the `failJob` method might be called while being in a `CANCELED`/`CANCELLING` state?For this case, I would still think that we should archive the exception because the users intervention happened after the exception happened.




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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r570428422



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
##########
@@ -1022,18 +1021,38 @@ private boolean transitionState(ExecutionState currentState, ExecutionState newS
         return transitionState(currentState, newState, null);
     }
 
+    /**
+     * Try to transition to FAILED state from a given state and sets the {@code failureCause}.
+     *
+     * @param currentState of the execution
+     * @param cause the {@link Throwable} causing the failure
+     * @return true if the transition was successful, otherwise false
+     * @throws NullPointerException if no {@code cause} is provided
+     */
+    private boolean transitionToFailedStateAndSetFailureCause(
+            ExecutionState currentState, Throwable cause) {
+        Preconditions.checkNotNull(cause, "No cause is given when transitioning to FAILED state.");
+        failureCause = cause;
+        return transitionState(currentState, ExecutionState.FAILED, cause);

Review comment:
       I think we can't update `failureCause` until we successfully transitioned to FAILED state.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 95f80b654417779460301440b21d1c4d1b9476e3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r567418341



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -641,6 +648,33 @@ public void cancel() {
         return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn());
     }
 
+    protected void archiveGlobalFailure(Throwable failure) {
+        Set<ExecutionAttemptID> executionAttemptIds =
+                IterableUtils.toStream(executionGraph.getAllExecutionVertices())
+                        .map(ExecutionVertex::getCurrentExecutionAttempt)
+                        .map(Execution::getAttemptId)
+                        .collect(Collectors.toSet());
+
+        globalTaskFailures.put(
+                executionAttemptIds, new ErrorInfo(failure, System.currentTimeMillis()));
+
+        log.debug("Archive global {}: {}", failure, failure.getMessage());
+    }
+
+    protected void archiveFromFailureHandlingResult(FailureHandlingResult failureHandlingResult) {
+        if (failureHandlingResult.getExecutionVertexIdOfFailedTask() == null) {

Review comment:
       I removed this check. This if was a relict from a previous code version.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cde4fbdc8d036cdb44884a0974fbf1cf31966c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912) 
   * 211a68e9532b0bf10581388b395e474123ec6ad5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944) 
   * ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a9204577129b7145846210351ee8754928287a3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cde4fbdc8d036cdb44884a0974fbf1cf31966c5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912) 
   * 211a68e9532b0bf10581388b395e474123ec6ad5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r570084862



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -522,6 +527,7 @@ protected ComponentMainThreadExecutor getMainThreadExecutor() {
     protected void failJob(Throwable cause) {
         incrementVersionsOfAllVertices();
         executionGraph.failJob(cause);
+        getTerminationFuture().thenRun(() -> archiveGlobalFailure(cause));

Review comment:
       I mean the case that a failure happens, the job goes into the `FAILING` state and tries to cancel the tasks and now the user cancels the job because it takes too long for him. Then the job will go into the `CANCELING` state which will result to the `CANCELED` state once all tasks have terminated.
   
   I think you are right that we should still record the failure 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:
users@infra.apache.org



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-778281634


   The test instability is caused by the change adding the requirement that `failureCause` need to be set if the `executionState` is `FAILED`. The failing test runs a task but doesn't wait for the task to be finished. The `TaskExecutor` triggers the cancellation of the `Task` through [Task.failExternally](https://github.com/XComp/flink/blob/354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L1129) passing in a `FlinkException: Disconnect from JobManager responsible for [...]`. This will trigger the `Task` to transition into `FAILED`. A context switch happens after the state is set but before the `failureCause` is set. Then, the `Task` finishes in the main thread, cleans up and calls [notifyFinalState](https://github.com/XComp/flink/blob/354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L904) as part of this which initializes a `TaskExecutionState` and run
 s into the `IllegalStateException` since the `failureCause` is still not set.
   
   The solution would be to do the state change and `failureCause` atomically. But I'm not sure whether this would be a performance problem. We would have to switch from [compareAndSet](https://github.com/XComp/flink/blob/354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L1064) to a normal lock, I guess. Alternatively, we could remove the invariant again and handle the `null` in the exception history. @tillrohrmann what are your thoughts on that?


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e962af6d8df77a725defeb92537837e6f685292f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487",
       "triggerID" : "e962af6d8df77a725defeb92537837e6f685292f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f05d87ce3393dca5b8df30919f439b8c272e982e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13503",
       "triggerID" : "f05d87ce3393dca5b8df30919f439b8c272e982e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * e962af6d8df77a725defeb92537837e6f685292f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487) 
   * f05d87ce3393dca5b8df30919f439b8c272e982e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13503) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478",
       "triggerID" : "854b645872b5596d2cc618c65dac0228c616eec2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e962af6d8df77a725defeb92537837e6f685292f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487",
       "triggerID" : "e962af6d8df77a725defeb92537837e6f685292f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 854b645872b5596d2cc618c65dac0228c616eec2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13478) 
   * e962af6d8df77a725defeb92537837e6f685292f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13487) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9a9204577129b7145846210351ee8754928287a3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620) 
   * 22d7f6d27419e0f58b738b10e3a05a64ef2f24dd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413",
       "triggerID" : "8e732bfb2bddc38ec7422f482dcda4be3d296408",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 8e732bfb2bddc38ec7422f482dcda4be3d296408 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13413) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     }, {
       "hash" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13321",
       "triggerID" : "94d299053103c1bec5dfefaf4089ed0b31c403bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337",
       "triggerID" : "b895dddd2a1833c10dd04f2823dd19e497fc882c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * b895dddd2a1833c10dd04f2823dd19e497fc882c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13337) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 95f80b654417779460301440b21d1c4d1b9476e3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276) 
   * 06115e6dc53dfd654db8ee33449352d8f43abd89 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284) 
   * 354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13284",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     }, {
       "hash" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285",
       "triggerID" : "354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 354eb574c98bbc5fdaedabc9f3ca7e4acfaa3746 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13285) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r569551171



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -317,13 +316,11 @@ public TaskManagerLocation getAssignedResourceLocation() {
                 : null;
     }
 
-    public Throwable getFailureCause() {
-        return failureCause;
-    }
-
     @Override
-    public String getFailureCauseAsString() {
-        return ExceptionUtils.stringifyException(getFailureCause());
+    public Optional<ErrorInfo> getFailureInfo() {
+        return failureCause == null
+                ? Optional.empty()
+                : Optional.of(new ErrorInfo(failureCause, getStateTimestamp(FAILED)));

Review comment:
       Yes, it does. I actually had it like that in a first version of the code but changed it for the sake of consistency after realizing that it's not Serializable (which was a problem in `ArchivedExecution`). I will revert it.




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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r567421044



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -267,9 +267,16 @@ private void restartTasksWithDelay(final FailureHandlingResult failureHandlingRe
         delayExecutor.schedule(
                 () ->
                         FutureUtils.assertNoException(
-                                cancelFuture.thenRunAsync(
-                                        restartTasks(executionVertexVersions, globalRecovery),
-                                        getMainThreadExecutor())),
+                                cancelFuture
+                                        .thenRunAsync(
+                                                () ->
+                                                        archiveFromFailureHandlingResult(
+                                                                failureHandlingResult),
+                                                getMainThreadExecutor())
+                                        .thenRunAsync(
+                                                restartTasks(
+                                                        executionVertexVersions, globalRecovery),
+                                                getMainThreadExecutor())),

Review comment:
       That's a good question. I did it when experimenting for convenience and didn't reiterate over it again. 👍 I refactored it now.




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



[GitHub] [flink] tillrohrmann edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
tillrohrmann edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-775012926


   Ok, then let's first fix this instability before merging this PR @XComp. Ping me once the instability has been fixed.


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r567417575



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResult.java
##########
@@ -100,17 +116,22 @@ public long getRestartDelayMS() {
         }
     }
 
+    /**
+     * Returns the {@link ExecutionVertexID} of the task causing this failure.
+     *
+     * @return The {@code ExecutionVertexID} or {@code null} if it's a global failure.
+     */
+    public ExecutionVertexID getExecutionVertexIdOfFailedTask() {

Review comment:
       I switched to returning an `Optional` to make this more explicit.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6278de3bae76474c390b6251551e68862e3841f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710) 
   * 0cde4fbdc8d036cdb44884a0974fbf1cf31966c5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] XComp commented on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-779221947


   I created [FLINK-21376](https://issues.apache.org/jira/browse/FLINK-21376) to cover the `failureCause`/`executionState` synchronization. The code changes of FLINK-21187 will handle cases where no `failureCause` is set for now.


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



[GitHub] [flink] XComp commented on a change in pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #14798:
URL: https://github.com/apache/flink/pull/14798#discussion_r569986337



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -522,6 +527,7 @@ protected ComponentMainThreadExecutor getMainThreadExecutor() {
     protected void failJob(Throwable cause) {
         incrementVersionsOfAllVertices();
         executionGraph.failJob(cause);
+        getTerminationFuture().thenRun(() -> archiveGlobalFailure(cause));

Review comment:
       Here, I don't understand fully: You're referring to the case where a failure happens, the user cancels the job while the failure handling is done and the `failJob` method might be called while being in a `CANCELED`/`CANCELLING` state?For this case, I would still think that we should archive the exception because the users intervention happened after the exception happened.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -870,6 +874,78 @@ public void allocationIsCanceledWhenVertexIsFailedOrCanceled() throws Exception
         assertThat(testExecutionSlotAllocator.getPendingRequests().keySet(), hasSize(0));
     }
 
+    @Test
+    public void testExceptionHistoryWithRestartableFailure() {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final JobID jobId = jobGraph.getJobID();
+
+        final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+        // initiate restartable failure
+        final ExecutionAttemptID restartableAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException restartableException = new RuntimeException("restartable exception");
+        Range<Long> updateStateTriggeringRestartTimeframe =
+                initiateFailure(scheduler, jobId, restartableAttemptId, restartableException);
+
+        taskRestartExecutor.triggerNonPeriodicScheduledTask();
+
+        // initiate job failure
+        testRestartBackoffTimeStrategy.setCanRestart(false);
+
+        final ExecutionAttemptID failingAttemptId =
+                Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices())
+                        .getCurrentExecutionAttempt()
+                        .getAttemptId();
+        final RuntimeException failingException = new RuntimeException("failing exception");
+        Range<Long> updateStateTriggeringJobFailureTimeframe =
+                initiateFailure(scheduler, jobId, failingAttemptId, failingException);
+
+        List<ErrorInfo> actualExceptionHistory = scheduler.getExceptionHistory();
+        assertThat(actualExceptionHistory.size(), is(2));
+
+        // assert restarted attempt
+        ErrorInfo restartableFailure = actualExceptionHistory.get(0);
+        assertThat(
+                restartableFailure
+                        .getException()
+                        .deserializeError(ClassLoader.getSystemClassLoader()),
+                is(restartableException));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringRestartTimeframe.lowerEndpoint()));
+        assertThat(
+                restartableFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringRestartTimeframe.upperEndpoint()));
+
+        // assert job failure attempt
+        ErrorInfo globalFailure = actualExceptionHistory.get(1);
+        Throwable actualException =
+                globalFailure.getException().deserializeError(ClassLoader.getSystemClassLoader());
+        assertThat(actualException, org.hamcrest.core.IsInstanceOf.instanceOf(JobException.class));
+        assertThat(actualException.getCause(), is(failingException));
+        assertThat(
+                globalFailure.getTimestamp(),
+                greaterThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.lowerEndpoint()));
+        assertThat(
+                globalFailure.getTimestamp(),
+                lessThanOrEqualTo(updateStateTriggeringJobFailureTimeframe.upperEndpoint()));
+    }
+
+    private static Range<Long> initiateFailure(
+            DefaultScheduler scheduler,
+            JobID jobId,
+            ExecutionAttemptID executionAttemptID,
+            Throwable exception) {
+        long start = System.currentTimeMillis();

Review comment:
       Thanks for clarification.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
##########
@@ -1022,18 +1021,38 @@ private boolean transitionState(ExecutionState currentState, ExecutionState newS
         return transitionState(currentState, newState, null);
     }
 
+    /**
+     * Try to transition to FAILED state from a given state and sets the {@code failureCause}.
+     *
+     * @param currentState of the execution
+     * @param cause the {@link Throwable} causing the failure
+     * @return true if the transition was successful, otherwise false
+     * @throws NullPointerException if no {@code cause} is provided
+     */
+    private boolean transitionToFailedStateAndSetFailureCause(
+            ExecutionState currentState, Throwable cause) {
+        Preconditions.checkNotNull(cause, "No cause is given when transitioning to FAILED state.");
+        failureCause = cause;
+        return transitionState(currentState, ExecutionState.FAILED, cause);

Review comment:
       That's actually a good point. I will correct that.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6278de3bae76474c390b6251551e68862e3841f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 22d7f6d27419e0f58b738b10e3a05a64ef2f24dd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621) 
   * b8f6541c7820d1eea9c29c915690070cfebe837b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14798: [FLINK-21187] Provide exception history for root causes

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14798:
URL: https://github.com/apache/flink/pull/14798#issuecomment-769223911


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9a9204577129b7145846210351ee8754928287a3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12620",
       "triggerID" : "9a9204577129b7145846210351ee8754928287a3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12621",
       "triggerID" : "22d7f6d27419e0f58b738b10e3a05a64ef2f24dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12644",
       "triggerID" : "b8f6541c7820d1eea9c29c915690070cfebe837b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6278de3bae76474c390b6251551e68862e3841f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12710",
       "triggerID" : "f6278de3bae76474c390b6251551e68862e3841f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12912",
       "triggerID" : "0cde4fbdc8d036cdb44884a0974fbf1cf31966c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12944",
       "triggerID" : "211a68e9532b0bf10581388b395e474123ec6ad5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12951",
       "triggerID" : "ab763747a3acaff1d72dfe9cd9bdbb3ed0896d70",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12962",
       "triggerID" : "5910b1a876d28886a8b5f87c09e67d75d2a45cd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005",
       "triggerID" : "4c8bc864776e0923683d1e60a1da38a12e9a52d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "af8de65a4905ae47ab704a75acfcd1b2e897d915",
       "triggerType" : "PUSH"
     }, {
       "hash" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "772e076ed9d773e16713236c942f4e30b659d6eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276",
       "triggerID" : "95f80b654417779460301440b21d1c4d1b9476e3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "06115e6dc53dfd654db8ee33449352d8f43abd89",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c8bc864776e0923683d1e60a1da38a12e9a52d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13005) 
   * af8de65a4905ae47ab704a75acfcd1b2e897d915 UNKNOWN
   * 772e076ed9d773e16713236c942f4e30b659d6eb UNKNOWN
   * 95f80b654417779460301440b21d1c4d1b9476e3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13276) 
   * 06115e6dc53dfd654db8ee33449352d8f43abd89 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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