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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -376,7 +376,7 @@ private void confirmLeadership(
                                     LOG.debug("Confirm leadership {}.", 
leaderSessionId);
                                     
leaderElection.confirmLeadership(leaderSessionId, address);
                                 } else {
-                                    LOG.trace(
+                                    LOG.debug(
                                             "Ignore confirming leadership 
because the leader {} is no longer valid.",
                                             leaderSessionId);
                                 }

Review Comment:
   the entire callback can be replaced by `runIfValidLeader`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -262,34 +262,67 @@ 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()),
+                                                () ->
+                                                        
FutureUtils.completedExceptionally(
+                                                                new 
LeadershipLostException(
+                                                                        "The 
leadership is lost.")))
+                                        .handle(
+                                                (hasJobResult, throwable) -> {
+                                                    if (throwable
+                                                            instanceof 
LeadershipLostException) {
+                                                        
printLogIfNotValidLeader(
+                                                                "verify job 
result entry",
+                                                                
leaderSessionId);
+                                                        return null;
+                                                    } else if (throwable != 
null) {
+                                                        
ExceptionUtils.rethrow(throwable);
+                                                    }
+                                                    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);
-            } else {
-                createNewJobMasterServiceProcess(leaderSessionId);
-            }
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format(
-                            "Could not retrieve the job scheduling status for 
job %s.", getJobID()),
-                    e);
-        }
+    private void handleJobAlreadyDoneIfValidLeader(UUID leaderSessionId) {
+        runIfValidLeader(
+                leaderSessionId,
+                () -> {
+                    jobAlreadyDone(leaderSessionId);
+                },
+                () -> printLogIfNotValidLeader("check completed job", 
leaderSessionId));
+    }
+
+    private void createNewJobMasterServiceProcessIfValidLeader(UUID 
leaderSessionId) {
+        runIfValidLeader(
+                leaderSessionId,
+                () -> {

Review Comment:
   obsolete brackets



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -489,19 +522,29 @@ private boolean isRunning() {
     }
 
     private void runIfValidLeader(

Review Comment:
   In most of the cases, we're just creating the log message as a fallback. We 
could reduce redundant code by adding the following helper method:
   ```
       private void runIfValidLeader(
               UUID expectedLeaderId, Runnable action, String 
noLeaderFallbackCommandDescription) {
           runIfValidLeader(
                   expectedLeaderId,
                   action,
                   () ->
                           printLogIfNotValidLeader(
                                   noLeaderFallbackCommandDescription, 
expectedLeaderId));
       }
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -392,7 +392,7 @@ private void forwardResultFuture(
                         if (isValidLeader(leaderSessionId)) {
                             onJobCompletion(jobManagerRunnerResult, throwable);
                         } else {
-                            LOG.trace(
+                            LOG.debug(

Review Comment:
   The entire callback can be replaced by `runIfValidLeader`.



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