zentol commented on code in PR #25679: URL: https://github.com/apache/flink/pull/25679#discussion_r1871005220
########## 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: is it a problem that this runs in the leadership operation thread now? I suppose not, but it's the one thing that stood out to me. -- 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