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

Gary Yao commented on FLINK-12302:
----------------------------------

Hi [~lamber-ken],

Thanks for the update. If I understand correctly, after the job transitions 
into a globally terminal state ({{FAILED}}), you kill the ApplicationMaster 
(AM). Because you kill the AM, the application cannot be de-registered from 
YARN. The new AM, which is brought up by YARN, sees in the 
{{RunningJobsRegistry}} that the job is already in a terminal state, and we run 
into the _"jobFinishedByOther()"_ code path [1]. Because the new AM currently 
cannot know whether the job finished successfully or failed, we chose 
{{UNKNOWN}} as Flink's internal application status, which in turn is mapped to 
YARN's UNDEFINED final application status. Note that there are also other 
places in the code where we use Flink's {{UNKNOWN}} application status [3]. The 
bottom line is that I think your fix is not enough to consistently set the 
application status. If your patch was applied, I think even successfully 
finished jobs, could show up as {{FAILED}}.

I wonder how severe this issue is for you, and how often it occurs? It seems to 
me that the AM has to be killed in a very specific moment in time to reproduce 
the behavior. Please correct me if I am wrong.

Moreover, I wonder if using {{FinalApplicationStatus.UNDEFINED}} is wrong, per 
se. The Javadoc for {{FinalApplicationStatus.UNDEFINED}} reads:
{noformat}
Undefined state when either the application has not yet finished
{noformat}
The term _"either"_ implies that there is an alternative meaning, i.e., it 
looks like the original author of the Javadoc forgot to finish the sentence. 
Also the fact that it can be set by the user, implies that it is a valid final 
status.

[1] 
[https://github.com/apache/flink/blob/58987dd16c7e8af36e935e811f716d2f843de5ca/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java#L243]
 
[2] 
[https://github.com/apache/flink/blob/58987dd16c7e8af36e935e811f716d2f843de5ca/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java#L494]
[3] 
[https://github.com/apache/flink/blob/58987dd16c7e8af36e935e811f716d2f843de5ca/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java#L226-L229]

> Fixed the wrong finalStatus of yarn application when application finished
> -------------------------------------------------------------------------
>
>                 Key: FLINK-12302
>                 URL: https://issues.apache.org/jira/browse/FLINK-12302
>             Project: Flink
>          Issue Type: Improvement
>          Components: Deployment / YARN
>    Affects Versions: 1.8.0
>            Reporter: lamber-ken
>            Assignee: lamber-ken
>            Priority: Minor
>              Labels: pull-request-available
>             Fix For: 1.9.0
>
>         Attachments: fix-bad-finalStatus.patch, flink-conf.yaml, 
> image-2019-04-23-19-56-49-933.png, image-2019-05-28-00-46-49-740.png, 
> image-2019-05-28-00-50-13-500.png, jobmanager-05-27.log, jobmanager-1.log, 
> jobmanager-2.log, screenshot-1.png, screenshot-2.png, 
> spslave4.bigdata.ly_23951, spslave5.bigdata.ly_20271, test.jar
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> flink job(flink-1.6.3) failed in per-job yarn cluste mode, the 
> resourcemanager of yarn rerun the job.
> when the job failed again, the application while finish, but the finalStatus 
> is +UNDEFINED,+  It's better to show state +FAILED+
> !image-2019-04-23-19-56-49-933.png!
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to