[ 
https://issues.apache.org/jira/browse/FLINK-22276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17321932#comment-17321932
 ] 

Zhilong Hong commented on FLINK-22276:
--------------------------------------

Thank you for pointing this out, [~mapohl]. I go over the job with 
[[email protected]] and figure this out. When TaskExecutor updates the 
Execution state to FAILED, it will call 
{{JobMasterPartitionTracker#stopTrackingAndReleasePartitions}} for the result 
partition of {{shuffleRead&Write (1/2)}}.

Failover 2 is caused by {{shuffleRead&Write (2/2)}}. When failover 2 occurs, it 
will first iterate over all the downstream regions of {{shuffleRead&Write 
(2/2)}}, i.e. {{write (1/2)}} and {{write (2/2)}}. Then when iterating the 
region of {{write (1/2)}}, it will check if its needed input result partition 
is available or not, and then adds {{shuffleRead&Write (1/2)}} to the regions 
to restart. Therefore there are two tasks for restart in the failover 2.
 

> ExceptionHistoryEntryExtractor throws fatal error when task failure
> -------------------------------------------------------------------
>
>                 Key: FLINK-22276
>                 URL: https://issues.apache.org/jira/browse/FLINK-22276
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.13.0
>            Reporter: Jin Xing
>            Assignee: Matthias
>            Priority: Blocker
>             Fix For: 1.13.0
>
>         Attachments: image-2021-04-14-17-50-45-199.png, log
>
>
> When running my batch job on Flink cluster, I got a fatal error as below and 
> JM exits:
> !image-2021-04-14-17-50-45-199.png!
> Digging into the code,   when DefaultScheduler start archiving failure cause 
> ([https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java#L259),]
>  seems Execution#failureCause is not safely/correctly attached/updated.
> I attached JM log, [~mapohl] Would you mind help verify on this ?



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to