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

Xin Chen commented on FLINK-33483:
----------------------------------

Hi [~mapohl] yes, this is not a question in recent version. But I also see some 
code about "ApplicationStatus.UNKNOWN" and "UNDEFINED", so it is confused 
whether there still be "UNDEFINED" scenarios in Flink?  Some other code like:

{code:java}
// ApplicationDispatcherBootstrap.java#getJobResult()
private CompletableFuture<JobResult> getJobResult(
            final DispatcherGateway dispatcherGateway,
            final JobID jobId,
            final ScheduledExecutor scheduledExecutor,
            final boolean tolerateMissingResult) {
        final Time timeout =
                
Time.milliseconds(configuration.get(ClientOptions.CLIENT_TIMEOUT).toMillis());
        final Time retryPeriod =
                
Time.milliseconds(configuration.get(ClientOptions.CLIENT_RETRY_PERIOD).toMillis());
        final CompletableFuture<JobResult> jobResultFuture =
                JobStatusPollingUtils.getJobResult(
                        dispatcherGateway, jobId, scheduledExecutor, timeout, 
retryPeriod);
        if (tolerateMissingResult) {
            // Return "unknown" job result if dispatcher no longer knows the 
actual result.
            return FutureUtils.handleException(
                    jobResultFuture,
                    FlinkJobNotFoundException.class,
                    exception ->
                            new JobResult.Builder()
                                    .jobId(jobId)
                                    
.applicationStatus(ApplicationStatus.UNKNOWN)
                                    .netRuntime(Long.MAX_VALUE)
                                    .build());
        }
        return jobResultFuture;
    }



// RestClusterClient.java#requestJobResultInternal
private CompletableFuture<JobResult> requestJobResultInternal(@Nonnull JobID 
jobId) {
        return pollResourceAsync(
                        () -> {
                            final JobMessageParameters messageParameters =
                                    new JobMessageParameters();
                            messageParameters.jobPathParameter.resolve(jobId);
                            return sendRequest(
                                    JobExecutionResultHeaders.getInstance(), 
messageParameters);
                        })
                .thenApply(
                        jobResult -> {
                            if (jobResult.getApplicationStatus() == 
ApplicationStatus.UNKNOWN) {
                                throw new JobStateUnknownException(
                                        String.format("Result for Job %s is 
UNKNOWN", jobId));
                            }
                            return jobResult;
                        });
    }
{code}

I want to know which other scenarios the final state of the Flink task may be 
UNDEFINED.

> Why is “UNDEFINED” defined in the Flink task status?
> ----------------------------------------------------
>
>                 Key: FLINK-33483
>                 URL: https://issues.apache.org/jira/browse/FLINK-33483
>             Project: Flink
>          Issue Type: Improvement
>          Components: Deployment / YARN
>    Affects Versions: 1.12.2
>            Reporter: Xin Chen
>            Priority: Major
>         Attachments: container_e15_1693914709123_8498_01_000001_8042, 
> reproduce.log
>
>
> In the Flink on Yarn mode, if an unknown status appears in the Flink log, 
> jm(jobmanager) will report the task status as undefined. The Yarn page will 
> display the state as FINISHED, but the final status is *UNDEFINED*. In terms 
> of business, it is unknown whether the task has failed or succeeded, and 
> whether to retry. It has a certain impact. Why should we design UNDEFINED? 
> Usually, this situation occurs due to zk(zookeeper) disconnection or jm 
> abnormality, etc. Since the abnormality is present, why not use FAILED?
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to