XComp commented on code in PR #28201:
URL: https://github.com/apache/flink/pull/28201#discussion_r3389797621


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -241,29 +254,83 @@ public boolean isInitialized() {
 
     @Override
     public void grantLeadership(UUID leaderSessionID) {
-        runIfStateRunning(
-                () -> startJobMasterServiceProcessAsync(leaderSessionID),
-                "starting a new JobMasterServiceProcess");
+        synchronized (lock) {
+            if (!isRunning()) {
+                LOG.debug(
+                        "Ignore 'starting a new JobMasterServiceProcess' 
because the leadership runner is no longer running.");
+                return;
+            }
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused -> 
flushPendingOrStartNewProcessAsync(leaderSessionID));
+            handleAsyncOperationError(sequentialOperation, "Could not start 
the job manager.");
+        }
+    }
+
+    private CompletableFuture<Void> flushPendingOrStartNewProcessAsync(UUID 
leaderSessionId) {
+        final JobManagerRunnerResult cachedTerminalResult;
+        synchronized (lock) {
+            if (!isRunning()) {
+                return FutureUtils.completedVoidFuture();
+            }
+            cachedTerminalResult = takePendingTerminalResult();
+            if (cachedTerminalResult != null) {
+                state = State.JOB_COMPLETED;
+            }
+        }
+
+        if (cachedTerminalResult != null) {
+            LOG.info(
+                    "Flushing previously observed globally terminal result for 
job {} on re-grant; not starting a new {}. Job state: {}.",
+                    getJobID(),
+                    JobMasterServiceProcess.class.getSimpleName(),
+                    cachedTerminalResult
+                            .getExecutionGraphInfo()
+                            .getArchivedExecutionGraph()
+                            .getState());
+            resultFuture.complete(cachedTerminalResult);
+            return FutureUtils.completedVoidFuture();
+        }
+
+        return jobResultStore
+                .hasJobResultEntryAsync(getJobID())
+                .thenCompose(
+                        hasJobResult ->
+                                hasJobResult
+                                        ? 
handleJobAlreadyDoneIfValidLeader(leaderSessionId)
+                                        : 
createNewJobMasterServiceProcessIfValidLeader(
+                                                leaderSessionId));
     }
 
     @GuardedBy("lock")
-    private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenCompose(
-                        unused ->
-                                jobResultStore
-                                        .hasJobResultEntryAsync(getJobID())
-                                        .thenCompose(
-                                                hasJobResult -> {
-                                                    if (hasJobResult) {
-                                                        return 
handleJobAlreadyDoneIfValidLeader(
-                                                                
leaderSessionId);
-                                                    } else {
-                                                        return 
createNewJobMasterServiceProcessIfValidLeader(
-                                                                
leaderSessionId);
-                                                    }
-                                                }));
-        handleAsyncOperationError(sequentialOperation, "Could not start the 
job manager.");
+    private JobManagerRunnerResult takePendingTerminalResult() {
+        final JobManagerRunnerResult terminalResult = pendingTerminalResult;
+        pendingTerminalResult = null;
+        if (terminalResult != null) {
+            currentJobMasterServiceProcessLeaderId = null;
+        }
+        return terminalResult;
+    }
+
+    private void completeResultFutureAfterClose() {
+        JobManagerRunnerResult closeResult;
+        synchronized (lock) {
+            closeResult = takePendingTerminalResult();
+            if (closeResult == null) {
+                closeResult =
+                        JobManagerRunnerResult.forSuccess(
+                                
createExecutionGraphInfoWithJobStatus(JobStatus.SUSPENDED));
+                currentJobMasterServiceProcessLeaderId = null;

Review Comment:
   we would have to reset `currentJobMasterServiceProcessLeaderId` to `null` in 
`flushPendingOrStartNewProcessAsync` as well to keep the class instance's state 
consistent



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to