WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1297932131
##########
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:
The brackets in `handleJobAlreadyDoneIfValidLeader` and
`createNewJobMasterServiceProcessIfValidLeader` have been removed.
--
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]