tillrohrmann commented on a change in pull request #16894: URL: https://github.com/apache/flink/pull/16894#discussion_r692914530
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobAlreadyDoneException.java ########## @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.JobException; + +/** Exception indicating that a Flink job failed in finishing phase. */ +public class JobAlreadyDoneException extends JobException { Review comment: Can this be package private? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java ########## @@ -659,6 +660,31 @@ public void testJobMasterServiceProcessCreationFailureIsForwardedToResultFuture( "Result future should be completed exceptionally.")); } + @Test + public void testJobAlreadyDone() throws Exception { + JobID jobID = new JobID(); + try (JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .setJobMasterServiceProcessFactory( + TestingJobMasterServiceProcessFactory.newBuilder() + .setJobId(jobID) + .build()) + .build()) { + runningJobsRegistry.setJobFinished(jobID); + jobManagerRunner.start(); + leaderElectionService.isLeader(UUID.randomUUID()); + + final CompletableFuture<JobManagerRunnerResult> resultFuture = + jobManagerRunner.getResultFuture(); + + JobManagerRunnerResult result = + resultFuture.get(TESTING_TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); Review comment: ```suggestion resultFuture.get(); ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java ########## @@ -285,7 +285,12 @@ private ExecutionGraphInfo createExecutionGraphInfoWithJobStatus(JobStatus jobSt } private void jobAlreadyDone() { - resultFuture.completeExceptionally(new JobNotFinishedException(getJobID())); + resultFuture.complete( + JobManagerRunnerResult.forSuccess( + new ExecutionGraphInfo( + jobMasterServiceProcessFactory.createArchivedExecutionGraph( + JobStatus.FAILED, + new JobAlreadyDoneException(getJobID()))))); Review comment: I think if we communicate that way to the `Dispatcher` that a job has been already terminated, then we don't need `JobManagerRunnerResult.isJobAlreadyDone()`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcess.java ########## @@ -214,15 +214,6 @@ public void jobReachedGloballyTerminalState(ExecutionGraphInfo executionGraphInf resultFuture.complete(JobManagerRunnerResult.forSuccess(executionGraphInfo)); } - @Override Review comment: I also couldn't find another user of this method. Hence, I believe it is ok to remove it. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java ########## @@ -446,9 +446,13 @@ private CleanupJobState handleJobManagerRunnerResult( JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { if (jobManagerRunnerResult.isInitializationFailure()) { if (executionType == ExecutionType.RECOVERY) { - return jobManagerRunnerFailed( - jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), - jobManagerRunnerResult.getInitializationFailure()); + if (jobManagerRunnerResult.isJobAlreadyDone()) { + return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); Review comment: With the current state of this PR, we probably don't need this branch. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobAlreadyDoneException.java ########## @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.JobException; + +/** Exception indicating that a Flink job failed in finishing phase. */ +public class JobAlreadyDoneException extends JobException { + private static final long serialVersionUID = 5906282500322681417L; + + public JobAlreadyDoneException(JobID jobId) { + super("The job (" + jobId + ") has already terminated."); Review comment: I think we should add that we cannot tell the final state and also not serve the `JobResult`. Consequently, the user would have to check the result of the job manually (e.g. whether output files are there). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org