XComp commented on code in PR #25679: URL: https://github.com/apache/flink/pull/25679#discussion_r1871483945
########## flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java: ########## @@ -256,45 +255,29 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) { sequentialOperation = sequentialOperation.thenCompose( unused -> - supplyAsyncIfValidLeader( - leaderSessionId, - () -> - 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); - } + jobResultStore + .hasJobResultEntryAsync(getJobID()) + .thenCompose( + hasJobResult -> { if (hasJobResult) { - handleJobAlreadyDoneIfValidLeader( + return handleJobAlreadyDoneIfValidLeader( leaderSessionId); } else { - createNewJobMasterServiceProcessIfValidLeader( + return createNewJobMasterServiceProcessIfValidLeader( leaderSessionId); } - return null; })); handleAsyncOperationError(sequentialOperation, "Could not start the job manager."); } - private void handleJobAlreadyDoneIfValidLeader(UUID leaderSessionId) { - runIfValidLeader( + private CompletableFuture<Void> handleJobAlreadyDoneIfValidLeader(UUID leaderSessionId) { + return runIfValidLeader( leaderSessionId, () -> jobAlreadyDone(leaderSessionId), "check completed job"); } - private void createNewJobMasterServiceProcessIfValidLeader(UUID leaderSessionId) { - runIfValidLeader( + private CompletableFuture<Void> createNewJobMasterServiceProcessIfValidLeader( + UUID leaderSessionId) { + return runIfValidLeader( Review Comment: ```java // the heavy lifting of the JobMasterServiceProcess instantiation is still // done asynchronously (see DefaultJobMasterServiceFactory#createJobMasterService // executing the logic on the leaderOperation thread in the DefaultLeaderElectionService // should be, therefore, fine ``` Yeah, that should be fine. I added a comment to the code to clarify this. One could think of making the interfaces for creating the `JobMasterService` more explicit. Moving the instantiation out of the leadership check should be fine as well. But I was hesitant to touch that now since having the instantiation trigger being protected by the leadership is something that was part of the code for a while. 🤔 -- 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