tillrohrmann commented on a change in pull request #16894:
URL: https://github.com/apache/flink/pull/16894#discussion_r692176396



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -578,6 +579,45 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() 
throws Exception {
         fatalErrorHandler.clearError();
     }
 
+    /** Tests that the {@link Dispatcher} do not fail by {@link 
JobFinishingException}. */

Review comment:
       ```suggestion
       /** Tests that the {@link Dispatcher} does not fail by {@link 
JobFinishingException}. */
   ```

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobFinishingException.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 JobFinishingException extends JobException {
+    private static final long serialVersionUID = 5906282500322681417L;
+
+    public JobFinishingException(JobID jobId) {
+        super("The job (" + jobId + ") is in finishing phase.");

Review comment:
       Maybe we should say that the job has already terminated and we cannot 
say what its final state was and also cannot return its `JobResult`.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobFinishingException.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 JobFinishingException extends JobException {

Review comment:
       Maybe 
   ```suggestion
   public class JobAlreadyDoneException extends JobException {
   ```
   is a slightly better name.

##########
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.isJobFinishing()) {
+                    return CleanupJobState.GLOBAL;

Review comment:
       I think we should call `jobReachedTerminalState` so that we create an 
`ArchivedExecutionGraph`.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -578,6 +579,45 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() 
throws Exception {
         fatalErrorHandler.clearError();
     }
 
+    /** Tests that the {@link Dispatcher} do not fail by {@link 
JobFinishingException}. */
+    @Test
+    public void testNonFatalErrorWhenRecoveredJobFinishingException() throws 
Exception {

Review comment:
       Maybe we should add another test similar to 
`DispatcherTest.testJobDataAreCleanedUpInCorrectOrder` that ensures that we 
clean up things properly.




-- 
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


Reply via email to