WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1296743085


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -262,36 +261,92 @@ public void grantLeadership(UUID leaderSessionID) {
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
         sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
+                sequentialOperation
+                        .thenCompose(
+                                unused ->
+                                        supplyAsyncIfValidLeader(
+                                                leaderSessionId,
                                                 () ->
-                                                        
verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                
leaderSessionId)),
-                                        "verify job scheduling status and 
create JobMasterServiceProcess"));
-
+                                                        
jobResultStore.hasJobResultEntryAsync(
+                                                                getJobID()),
+                                                () ->
+                                                        
printLogIfNotValidLeader(
+                                                                "verify job 
result entry",
+                                                                
leaderSessionId)))
+                        .handle(
+                                (hasJobResult, throwable) -> {
+                                    if (throwable != null) {
+                                        ExceptionUtils.rethrow(throwable);
+                                    }
+                                    if (hasJobResult == null) {
+                                        return null;
+                                    }
+                                    if (hasJobResult) {
+                                        
handleJobAlreadyDoneIfValidLeader(leaderSessionId);
+                                    } else {
+                                        
createNewJobMasterServiceProcessIfValidLeader(
+                                                leaderSessionId);
+                                    }
+                                    return null;
+                                });
         handleAsyncOperationError(sequentialOperation, "Could not start the 
job manager.");
     }
 
-    @GuardedBy("lock")
-    private void 
verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(UUID leaderSessionId)
-            throws FlinkException {
-        try {
-            if (jobResultStore.hasJobResultEntry(getJobID())) {
-                jobAlreadyDone(leaderSessionId);
+    private <T> CompletableFuture<T> supplyAsyncIfValidLeader(
+            UUID expectedLeaderId,
+            Supplier<CompletableFuture<T>> supplier,
+            Supplier<CompletableFuture<T>> noLeaderFallback) {
+        final CompletableFuture<T> resultFuture = new CompletableFuture<>();
+        runIfValidLeader(
+                expectedLeaderId,
+                () -> FutureUtils.forward(supplier.get(), resultFuture),
+                () -> FutureUtils.forward(noLeaderFallback.get(), 
resultFuture));
+
+        return resultFuture;
+    }
+
+    private void handleJobAlreadyDoneIfValidLeader(UUID leaderSessionId) {
+        runIfValidLeader(
+                leaderSessionId,
+                () -> {
+                    jobAlreadyDone(leaderSessionId);
+                },
+                () -> printLogIfNotValidLeader("check completed job", 
leaderSessionId));
+    }
+
+    private void createNewJobMasterServiceProcessIfValidLeader(UUID 
leaderSessionId) {
+        runIfValidLeader(
+                leaderSessionId,
+                () -> {
+                    ThrowingRunnable.unchecked(
+                                    () -> 
createNewJobMasterServiceProcess(leaderSessionId))
+                            .run();
+                },
+                () ->
+                        printLogIfNotValidLeader(
+                                "create new job master service process", 
leaderSessionId));
+    }
+
+    private void runIfValidLeader(
+            UUID expectedLeaderId, Runnable action, Runnable noLeaderFallback) 
{
+        synchronized (lock) {
+            if (isValidLeader(expectedLeaderId)) {
+                action.run();
             } else {
-                createNewJobMasterServiceProcess(leaderSessionId);
+                noLeaderFallback.run();
             }
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format(
-                            "Could not retrieve the job scheduling status for 
job %s.", getJobID()),
-                    e);
         }
     }
 
+    private CompletableFuture<Boolean> printLogIfNotValidLeader(
+            String actionDescription, UUID leaderSessionId) {
+        LOG.trace(

Review Comment:
   Thanks for the suggestion! I've followed the 2nd solution.



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