You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "WencongLiu (via GitHub)" <gi...@apache.org> on 2023/04/04 06:48:31 UTC

[GitHub] [flink] WencongLiu opened a new pull request, #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

WencongLiu opened a new pull request, #22341:
URL: https://github.com/apache/flink/pull/22341

   ## What is the purpose of the change
   
   *At present, FileSystemJobResultStore executes I/O operations through FileSystem directly. We should refract the interface of JobResultStore to make I/O operations be executed asynchronously. This would move the responsibility of I/O operation from the Dispatcher into the JobResultStore.*
   
   
   ## Brief change log
   
     - *Refract the JobResultStore interface.*
     - *Refract all codes that calls JobResultStore.*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1550846620

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1212817671


##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -80,33 +82,65 @@ private TestingJobResultStore(
     }
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        createDirtyResultConsumer.accept(jobResultEntry);
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        try {
+            createDirtyResultConsumer.accept(jobResultEntry);
+        } catch (IOException e) {
+            CompletableFuture<Void> future = new CompletableFuture<>();
+            future.completeExceptionally(e);
+            return future;
+        }
+        return CompletableFuture.completedFuture(null);
     }
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException {
-        markResultAsCleanConsumer.accept(jobId);
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        try {
+            markResultAsCleanConsumer.accept(jobId);
+        } catch (IOException e) {

Review Comment:
   I've fixed the builder of TestingJobResultStore.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -44,106 +45,140 @@ public interface JobResultStoreContractTest {
     JobResultStore createJobResultStore() throws IOException;
 
     @Test
-    default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
+    default void testStoreJobResultsWithDuplicateIDsThrowsException()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .get())
+                .hasCauseInstanceOf(RuntimeException.class);
     }
 
     @Test
-    default void testStoreDirtyEntryForAlreadyCleanedJobResultThrowsException() throws IOException {
+    default void testStoreDirtyEntryForAlreadyCleanedJobResultThrowsException()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY))
-                .isInstanceOf(IllegalStateException.class);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get();
+        assertThatThrownBy(
+                        () -> jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get())
+                .hasCauseInstanceOf(RuntimeException.class);
     }
 
     @Test
-    default void testCleaningDuplicateEntryThrowsNoException() throws IOException {
+    default void testCleaningDuplicateEntryThrowsNoException()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get();
         assertThatNoException()
                 .isThrownBy(
-                        () -> jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()));
+                        () ->
+                                jobResultStore
+                                        .markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                        .get());
     }
 
     @Test
     default void testCleaningNonExistentEntryThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
         assertThatThrownBy(
-                        () -> jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()))
-                .isInstanceOf(NoSuchElementException.class);
+                        () ->
+                                jobResultStore
+                                        .markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                        .get())
+                .hasCauseInstanceOf(NoSuchElementException.class);
     }
 
     @Test
-    default void testHasJobResultEntryWithDirtyEntry() throws IOException {
+    default void testHasJobResultEntryWithDirtyEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        assertThat(jobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        assertThat(
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
                 .isTrue();
-        assertThat(jobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        assertThat(
+                        jobResultStore
+                                .hasCleanJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
                 .isFalse();
-        assertThat(jobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())).isTrue();
+        assertThat(jobResultStore.hasJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get())
+                .isTrue();
     }
 
     @Test
-    default void testHasJobResultEntryWithCleanEntry() throws IOException {
+    default void testHasJobResultEntryWithCleanEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-        assertThat(jobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get();
+        assertThat(
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
                 .isFalse();
-        assertThat(jobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        assertThat(
+                        jobResultStore
+                                .hasCleanJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
+                .isTrue();
+        assertThat(jobResultStore.hasJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get())
                 .isTrue();
-        assertThat(jobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())).isTrue();
     }
 
     @Test
-    default void testHasJobResultEntryWithEmptyStore() throws IOException {
+    default void testHasJobResultEntryWithEmptyStore()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
         JobID jobId = new JobID();
-        assertThat(jobResultStore.hasDirtyJobResultEntry(jobId)).isFalse();
-        assertThat(jobResultStore.hasCleanJobResultEntry(jobId)).isFalse();
-        assertThat(jobResultStore.hasJobResultEntry(jobId)).isFalse();
+        assertThat(jobResultStore.hasDirtyJobResultEntryAsync(jobId).get()).isFalse();
+        assertThat(jobResultStore.hasCleanJobResultEntryAsync(jobId).get()).isFalse();
+        assertThat(jobResultStore.hasJobResultEntryAsync(jobId).get()).isFalse();
     }
 
     @Test
-    default void testGetDirtyResultsWithNoEntry() throws IOException {
+    default void testGetDirtyResultsWithNoEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        assertThat(jobResultStore.getDirtyResults()).isEmpty();
+        assertThat(jobResultStore.getDirtyResultsAsync().get()).isEmpty();
     }
 
     @Test
-    default void testGetDirtyResultsWithDirtyEntry() throws IOException {
+    default void testGetDirtyResultsWithDirtyEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
         assertThat(
-                        jobResultStore.getDirtyResults().stream()
+                        jobResultStore.getDirtyResultsAsync().get().stream()
                                 .map(JobResult::getJobId)
                                 .collect(Collectors.toList()))
                 .singleElement()
                 .isEqualTo(DUMMY_JOB_RESULT_ENTRY.getJobId());
     }
 
     @Test
-    default void testGetDirtyResultsWithDirtyAndCleanEntry() throws IOException {
+    default void testGetDirtyResultsWithDirtyAndCleanEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264367896


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1251,7 +1253,20 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
         if (cleanupJobState.isGlobalCleanup()) {
             return globalResourceCleaner
                     .cleanupAsync(jobId)
-                    .thenRunAsync(() -> markJobAsClean(jobId), ioExecutor)
+                    .thenRunAsync(

Review Comment:
   Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1262,14 +1277,8 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
         }
     }
 
-    private void markJobAsClean(JobID jobId) {
-        try {
-            jobResultStore.markResultAsClean(jobId);
-            log.debug(
-                    "Cleanup for the job '{}' has finished. Job has been marked as clean.", jobId);
-        } catch (IOException e) {
-            log.warn("Could not properly mark job {} result as clean.", jobId, e);
-        }
+    private CompletableFuture<Void> markJobAsCleanAsync(JobID jobId) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283947288


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -46,72 +47,97 @@ public interface JobResultStoreContractTest {
     @Test
     default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .join())
+                .isInstanceOf(CompletionException.class);

Review Comment:
   Fixed



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1282978842


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -516,7 +517,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
 
         try {
             if (isDuplicateJob(jobGraph.getJobID())) {
-                if (isInGloballyTerminalState(jobGraph.getJobID())) {
+                if (isInGloballyTerminalState(jobGraph.getJobID()).get()) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288176936


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,66 +46,92 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
 
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
+        this.ioExecutor = ioExecutor;
+    }
+
+    @Override
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .thenAccept(
+                        hasJobResultEntry ->
+                                Preconditions.checkState(
+                                        !hasJobResultEntry,
+                                        "Job result store already contains an entry for job %s",
+                                        jobResultEntry.getJobId()))
+                .thenCompose(
+                        ignoredVoid ->
+                                withWriteLockAsync(
+                                        () -> createDirtyResultInternal(jobResultEntry)));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);
-            return;
-        }
-
-        withWriteLock(() -> markResultAsCleanInternal(jobId));
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        return hasCleanJobResultEntryAsync(jobId)
+                .thenCompose(
+                        hasCleanJobResultEntry -> {
+                            if (hasCleanJobResultEntry) {
+                                LOG.debug(
+                                        "The job {} is already marked as clean. No action required.",
+                                        jobId);
+                                return FutureUtils.completedVoidFuture();
+                            }
+
+                            return withWriteLockAsync(() -> markResultAsCleanInternal(jobId));
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void markResultAsCleanInternal(JobID jobId)
             throws IOException, NoSuchElementException;
 
     @Override
-    public boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(
+    public CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(
                 () ->
                         hasDirtyJobResultEntryInternal(jobId)
                                 || hasCleanJobResultEntryInternal(jobId));
     }
 
     @Override
-    public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasDirtyJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasDirtyJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
-    public boolean hasCleanJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasCleanJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasCleanJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
     public Set<JobResult> getDirtyResults() throws IOException {
-        return withReadLock(this::getDirtyResultsInternal);
+        return getDirtyResultsInternal();
     }
 
     @GuardedBy("readWriteLock")
     protected abstract Set<JobResult> getDirtyResultsInternal() throws IOException;
 
+    private CompletableFuture<Void> withWriteLockAsync(ThrowingRunnable<IOException> runnable) {
+        return FutureUtils.supplyAsync(

Review Comment:
   Currently there are no `runAsync` method in org.apache.flink.util.concurrent.FutureUtils. I've added a new `runAsync` method in FutureUtils.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295337443


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {
+                                                            return null;
+                                                        }
+                                                        if (hasJobResult) {
+                                                            runIfValidLeader(
+                                                                    leaderSessionId,
+                                                                    () -> {
+                                                                        jobAlreadyDone(
+                                                                                leaderSessionId);
+                                                                        return CompletableFuture
+                                                                                .completedFuture(
+                                                                                        null);
+                                                                    },
+                                                                    "check completed job");
+                                                        } else {
+                                                            runIfValidLeader(
+                                                                    leaderSessionId,
+                                                                    () -> {
+                                                                        ThrowingRunnable.unchecked(
+                                                                                        () ->
+                                                                                                createNewJobMasterServiceProcess(
+                                                                                                        leaderSessionId))
+                                                                                .run();
+                                                                        return CompletableFuture
+                                                                                .completedFuture(
+                                                                                        null);
+                                                                    },
+                                                                    "create new job master service process");
+                                                        }
+                                                        return null;
+                                                    }));
+        } else {
+            LOG.trace(
+                    "Ignore leader action '{}' because the leadership runner is no longer the valid leader for {}.",
+                    "verify job scheduling status and create JobMasterServiceProcess",
+                    leaderSessionId);
+        }
         handleAsyncOperationError(sequentialOperation, "Could not start the job manager.");
     }
 
     @GuardedBy("lock")
     private void verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(UUID leaderSessionId)

Review Comment:
   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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1298451802


##########
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:
   You missed `forwardIfValidLeader`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1206880746


##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -80,33 +82,65 @@ private TestingJobResultStore(
     }
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        createDirtyResultConsumer.accept(jobResultEntry);
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        try {
+            createDirtyResultConsumer.accept(jobResultEntry);
+        } catch (IOException e) {
+            CompletableFuture<Void> future = new CompletableFuture<>();
+            future.completeExceptionally(e);
+            return future;
+        }
+        return CompletableFuture.completedFuture(null);
     }
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException {
-        markResultAsCleanConsumer.accept(jobId);
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        try {
+            markResultAsCleanConsumer.accept(jobId);
+        } catch (IOException e) {

Review Comment:
   You're not following the contract here, I guess. :thinking: Usually for these `Testing*` implementations, we would change the callback type (e.g. a `ThrowingConsumer<T>` becomes a `Function<T, CompletableFuture<Void>`). That enables us to have the entire logic being specified in the test method.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1263411729


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created
+     *     successfully, otherwise will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry);

Review Comment:
   Why did we change from `CompletableFuture<Void>` to `CompletableFuture<Boolean>`? Boolean doesn't add any value here. Or am I missing something :thinking: 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created

Review Comment:
   ```suggestion
        * @return a successfully completed future with {@code true} if the dirty result is created
   ```
   nit



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########


Review Comment:
   `markResultAsCleanInternal` has an obsolete `IOException` declaration in the method signature



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,87 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {

Review Comment:
   There's a `ExceptionUtils.tryRethrowException` method that's used to handle the case of `error` not being null. `hasResult` being false can be kept in the Precondition that's also used in the original code to reduce the diff. WDYT?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));

Review Comment:
   We shouldn't swallow the `InteruptedException`. Instead, we should call `Thread.currentThread().interrupt()` to continue the initiated interruption.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));
                                                     }
+                                                    result.complete(true);
+                                                    return result;

Review Comment:
   ```suggestion
                                                       return CompletableFuture.completedFuture(true);
   ```
   Even though, I'm still not sure whether we need a `boolean` return value (in that case `FutureUtils.completedVoidFuture()` would be a valid factory method for creating the return value.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -548,7 +557,10 @@ public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exce
                 TestingJobResultStore.builder()
                         .withCreateDirtyResultConsumer(
                                 jobResult -> {
-                                    throw new IOException("Expected IOException.");
+                                    CompletableFuture<Boolean> future = new CompletableFuture<>();
+                                    future.completeExceptionally(
+                                            new IOException("Expected IOException."));
+                                    return future;

Review Comment:
   ```suggestion
                                       return FutureUtils.completedExceptionally(new IOException("Expected IOException."))
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1262,14 +1277,8 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
         }
     }
 
-    private void markJobAsClean(JobID jobId) {
-        try {
-            jobResultStore.markResultAsClean(jobId);
-            log.debug(
-                    "Cleanup for the job '{}' has finished. Job has been marked as clean.", jobId);
-        } catch (IOException e) {
-            log.warn("Could not properly mark job {} result as clean.", jobId, e);
-        }
+    private CompletableFuture<Void> markJobAsCleanAsync(JobID jobId) {

Review Comment:
   You would rather have a `thenRun` in this method to trigger the debug log message.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -575,12 +576,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      */
     private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
         try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
+            return jobResultStore.hasJobResultEntryAsync(jobId).get();

Review Comment:
   This hasn't been addressed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -191,8 +191,8 @@ private Collection<JobResult> getDirtyJobResultsIfRunning() {
 
     private Collection<JobResult> getDirtyJobResults() {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   I added a related comment in `JobDispatcherLeaderProcessFactoryFactory`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created
+     *     successfully, otherwise will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
+     * @return CompletableFuture is the future with the completed state, which will throw {@link

Review Comment:
   ```suggestion
        * @return the future with the completed state, which will complete exceptionally with {@link
   ```
   That's a nitty one as well, but: The future doesn't throw (implying that it's an actor) but rather fails (being a passive component). The future's `get()` call would through the exception.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,37 +188,70 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
-        return fileSystem.exists(constructDirtyPath(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryInternal(JobID jobId) {
+        CompletableFuture<Boolean> hasDirtyJobResultEntryFuture = new CompletableFuture<>();
+        ioExecutor.execute(
+                () -> {
+                    try {
+                        hasDirtyJobResultEntryFuture.complete(
+                                fileSystem.exists(constructDirtyPath(jobId)));
+                    } catch (IOException e) {
+                        hasDirtyJobResultEntryFuture.completeExceptionally(e);
+                    }
+                });
+        return hasDirtyJobResultEntryFuture;

Review Comment:
   ```suggestion
   return FutureUtils.supplyAsync(
                   () -> fileSystem.exists(constructDirtyPath(jobId)), ioExecutor);        
   ```
   There's a shorter version for these kind of code constructrs.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -101,14 +109,9 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
         return new JobDispatcherLeaderProcessFactoryFactory(jobGraphRetriever);
     }
 
-    private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
-        try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
-            throw new FlinkRuntimeException(
-                    "Could not retrieve the JobResults of dirty jobs from the underlying JobResultStore.",
-                    e);
-        }
+    private static CompletableFuture<Set<JobResult>> getDirtyJobResultsAsync(

Review Comment:
   This method doesn't really have a usecase anymore. We're just calling `jobResultStore.getDirtyResultsAsync()` without any extra logic.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));

Review Comment:
   What's the reason for adding the `RuntimeException` as a wrapper here?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -77,7 +78,14 @@ public JobDispatcherLeaderProcessFactory createFactory(
         }
 
         final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore();
-        final Collection<JobResult> recoveredDirtyJobResults = getDirtyJobResults(jobResultStore);
+        Collection<JobResult> recoveredDirtyJobResults;

Review Comment:
   ```suggestion
           final Collection<JobResult> recoveredDirtyJobResults;
   ```
   The `final` keyword can stay



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,37 +188,70 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
-        return fileSystem.exists(constructDirtyPath(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryInternal(JobID jobId) {
+        CompletableFuture<Boolean> hasDirtyJobResultEntryFuture = new CompletableFuture<>();
+        ioExecutor.execute(
+                () -> {
+                    try {
+                        hasDirtyJobResultEntryFuture.complete(
+                                fileSystem.exists(constructDirtyPath(jobId)));
+                    } catch (IOException e) {
+                        hasDirtyJobResultEntryFuture.completeExceptionally(e);
+                    }
+                });
+        return hasDirtyJobResultEntryFuture;
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
-        return fileSystem.exists(constructCleanPath(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryInternal(JobID jobId) {
+        CompletableFuture<Boolean> hasCleanJobResultEntryFuture = new CompletableFuture<>();

Review Comment:
   Here you can use `FutureUtils.supplyAsync` as well



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -579,10 +591,17 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws
         final CompletableFuture<JobResultEntry> dirtyJobFuture = new CompletableFuture<>();
         final JobResultStore jobResultStore =
                 TestingJobResultStore.builder()
-                        .withCreateDirtyResultConsumer(dirtyJobFuture::complete)
+                        .withCreateDirtyResultConsumer(
+                                jobResultEntry -> {
+                                    dirtyJobFuture.complete(jobResultEntry);
+                                    return CompletableFuture.completedFuture(true);
+                                })
                         .withMarkResultAsCleanConsumer(
                                 jobId -> {
-                                    throw new IOException("Expected IOException.");
+                                    CompletableFuture<Void> result = new CompletableFuture<>();
+                                    result.completeExceptionally(
+                                            new IOException("Expected IOException."));
+                                    return result;

Review Comment:
   ```suggestion
                                       return FutureUtils.completedExceptionally(new IOException("Expected IOException."));
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -358,7 +363,11 @@ public void testJobBeingMarkedAsCleanAfterCleanup() throws Exception {
 
         final JobResultStore jobResultStore =
                 TestingJobResultStore.builder()
-                        .withMarkResultAsCleanConsumer(markAsCleanFuture::complete)
+                        .withMarkResultAsCleanConsumer(
+                                jobID -> {
+                                    markAsCleanFuture.complete(jobID);
+                                    return CompletableFuture.completedFuture(null);

Review Comment:
   nit: `FutureUtils.completedVoidFuture()` doesn't create a new instance. That doesn't make, admittedly, a difference in this test implementation. I wanted to mention it, anyway.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created
+     *     successfully, otherwise will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
+     * @return CompletableFuture is the future with the completed state, which will throw {@link
+     *     NoSuchElementException} if there is no corresponding {@code dirty} job present in the
      *     store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return CompletableFuture with {@code true} if a {@code dirty} or {@code clean} {@link
+     *     JobResultEntry} exists for the given {@code JobID}; otherwise {@code false}.
      */
-    default boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId);
+    default CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return hasDirtyJobResultEntryAsync(jobId)
+                .thenCombine(
+                        hasCleanJobResultEntryAsync(jobId),
+                        (result1, result2) -> result1 || result2);
     }
 
     /**
-     * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code dirty} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return CompletableFuture with value of {@code true}, if a {@code dirty} entry exists for the
+     *     given {@code JobID}; otherwise Completable with value of {@code false}.
      */
-    boolean hasDirtyJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains a {@code clean} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code clean} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code clean} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return CompletableFuture with value of {@code true}, if a {@code clean} entry exists for the
+     *     given {@code JobID}; otherwise Completable with value of {@code false}.
      */
-    boolean hasCleanJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId);
 
     /**
-     * Get the persisted {@link JobResult} instances that are marked as {@code dirty}. This is
-     * useful for recovery of finalization steps.
+     * Returns the future of persisted {@link JobResult} instances that are marked as {@code dirty}.
+     * This is useful for recovery of finalization steps.
      *
-     * @return A set of dirty {@code JobResults} from the store.
-     * @throws IOException if collecting the set of dirty results failed for IO reasons.
+     * @return CompletableFuture with value of a set of dirty {@code JobResults} from the store.
      */
-    Set<JobResult> getDirtyResults() throws IOException;
+    CompletableFuture<Set<JobResult>> getDirtyResultsAsync();

Review Comment:
   Just to repeat what I already said in another comment: Looks like we can revert making this method asynchronous.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1251,7 +1253,20 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
         if (cleanupJobState.isGlobalCleanup()) {
             return globalResourceCleaner
                     .cleanupAsync(jobId)
-                    .thenRunAsync(() -> markJobAsClean(jobId), ioExecutor)
+                    .thenRunAsync(

Review Comment:
   You could use `thenComposeAsync` here to avoid the `.get()` call. I'm wondering why we're using `Async here instead of just `thenRun` :thinking: 



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreContractTest.java:
##########
@@ -35,6 +35,6 @@ public class FileSystemJobResultStoreContractTest implements JobResultStoreContr
     @Override
     public JobResultStore createJobResultStore() throws IOException {
         Path path = new Path(temporaryFolder.toURI());
-        return new FileSystemJobResultStore(path.getFileSystem(), path, false);
+        return new FileSystemJobResultStore(path.getFileSystem(), path, false, Runnable::run);

Review Comment:
   nit: `Executors.directExecutor()` would be the usual Executor for direct execution.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -54,11 +58,19 @@ public class FileSystemJobResultStoreFileOperationsTest {
 
     private Path basePath;
 
+    private ExecutorService ioExecutor;
+
     @BeforeEach
     public void setupTest() throws IOException {
         basePath = new Path(temporaryFolder.toURI());
+        ioExecutor = Executors.newSingleThreadExecutor();

Review Comment:
   I guess, for the test cases in this class, the `directExecutor` is good enough (rather than creating a new thread). You could also use `ManuallyTriggeredScheduledExecutor` if you want to verify delay in the execution.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,84 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            try {
+                                withWriteLock(() -> createDirtyResultInternal(jobResultEntry));

Review Comment:
   Can't we move the asynchronous handling into `AbstractThreadsafeJobResultStore`. The `AbstractThreadsafeJobResultStore` would need the `executor` to be passed as a parameter in its constructor. The `EmbeddedJobResultStore` could then use `Executors.directExecutor()` which essentially handles the execution synchronously. WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -103,8 +103,8 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
 
     private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Here, I unfortunately misguided you: You just moved the error handling out of this method. Instead, the calling code does the error handling (`getDirtyJobResults` becomes obsoletes as part of this refactoring because it just calls a single method).
   
   The calling code, indeed, has to wait for the dirty results to be returned synchronously. I'm wondering whether we should simplify the `JobResultStore` interface here and revert the async refactoring for this specific method.
   
   Thinking about it, we could even separate the interfaces here: The `getDirtyJobResult` is used in the `DispatcherLeaderProcessFactoryFactory` implementations (something like `DirtyJobResultRetriever`). The other (now asynchronous) methods are used in the `Dispatcher` where we would want to have asynchronous access (`JobResultStoreAsync`). Anyway, I guess, that's not really necessary in this PR now. But reverting `getDirtyJobResult` appears to be reasonable. WDYT?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1290977544


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,17 +260,34 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);

Review Comment:
   We still need to use the `runIfValidLeader`. But we need to adapt it in a way that we also allow the returning of a value (for the `hasJobResultEntryAsync` return).



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,17 +260,34 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);

Review Comment:
   That goes into the right direction. :+1:  The leader validity should be checked when calling the methods (i.e. when checking the `jobResultStore` in the `thenCompose` and the handling of the response in the `handle` method).



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283100724


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1390,29 +1392,71 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
+        CompletableFuture<Boolean> shouldCheckDirtyJobResult =
+                jobResultStore
+                        .hasCleanJobResultEntryAsync(jobId)
+                        .handleAsync(
+                                (hasCleanJobResultEntry, throwable) -> {
+                                    if (throwable != null) {
+                                        writeFuture.completeExceptionally(throwable);
+                                        return false;
+                                    } else {
+                                        if (hasCleanJobResultEntry) {
+                                            log.warn(
+                                                    "Job {} is already marked as clean but "
+                                                            + "clean up was triggered again.",
+                                                    jobId);
+                                            writeFuture.complete(null);
+                                            return false;
+                                        } else {
+                                            return true;
+                                        }
+                                    }
+                                });
+        shouldCheckDirtyJobResult.whenCompleteAsync(
+                (shouldCheck, throwable1) -> {
+                    if (throwable1 != null) {
+                        writeFuture.completeExceptionally(throwable1);
                         return;
                     }
-                    writeFuture.complete(null);
+                    if (shouldCheck) {
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(jobId)
+                                .whenCompleteAsync(
+                                        (hasDirtyJobResultEntry, throwable2) -> {
+                                            if (throwable2 != null) {
+                                                writeFuture.completeExceptionally(throwable2);
+                                                return;
+                                            }
+                                            if (!hasDirtyJobResultEntry) {
+                                                jobResultStore
+                                                        .createDirtyResultAsync(
+                                                                new JobResultEntry(
+                                                                        JobResult.createFrom(
+                                                                                archivedExecutionGraph)))
+                                                        .whenCompleteAsync(
+                                                                (unused, throwable3) -> {
+                                                                    if (throwable3 != null) {
+                                                                        writeFuture
+                                                                                .completeExceptionally(
+                                                                                        throwable3);
+                                                                        return;
+                                                                    }
+                                                                    log.info(
+                                                                            "Job {} has been registered "
+                                                                                    + "for cleanup in "
+                                                                                    + "the JobResultStore "
+                                                                                    + "after reaching a "
+                                                                                    + "terminal state.",
+                                                                            jobId);
+                                                                    writeFuture.complete(null);
+                                                                });
+                                            } else {
+                                                writeFuture.complete(null);
+                                            }
+                                        });
+                    }

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1289452304


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,80 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        assertThat(emptyBaseDirectory).doesNotExist();
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
         assertThat(emptyBaseDirectory).exists().isDirectory();
     }
 
     @Test
     public void testStoreDirtyJobResultCreatesFile() throws Exception {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).doesNotExist();
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
         assertThat(getCleanResultIdsFromFileSystem()).isEmpty();
         assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).exists().isFile().isNotEmpty();
     }
 
     @Test
     public void testStoreCleanJobResultCreatesFile() throws Exception {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
+        CompletableFuture<Void> markCleanAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        assertThat(getCleanResultIdsFromFileSystem())
+                .doesNotContain(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(markCleanAsync).eventuallySucceeds();
         assertThat(getCleanResultIdsFromFileSystem())
                 .containsExactlyInAnyOrder(DUMMY_JOB_RESULT_ENTRY.getJobId());
     }
 
     @Test
-    public void testStoreCleanJobResultDeletesDirtyFile() throws Exception {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+    public void testStoreCleanJobResultDeletesDirtyFile() {
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).doesNotExist();
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
         assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).exists().isFile().isNotEmpty();
 
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        CompletableFuture<Void> markResultAsCleanAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(markResultAsCleanAsync).eventuallySucceeds();
         assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).doesNotExist();
     }
 
     @Test
     public void testCleanDirtyJobResultTwiceIsIdempotent() throws IOException {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
+        CompletableFuture<Void> cleanResultAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(cleanResultAsync).eventuallySucceeds();
         final byte[] cleanFileData =
                 FileUtils.readAllBytes(expectedCleanFile(DUMMY_JOB_RESULT_ENTRY).toPath());
 
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        CompletableFuture<Void> markResultAsCleanAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        assertThat(expectedCleanFile(DUMMY_JOB_RESULT_ENTRY))
+                .doesNotHaveSameHashCodeAs(cleanFileData);

Review Comment:
   I've added a fixup to remove the assert.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1293129190


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {

Review Comment:
   ```suggestion
   ```
   This part is obsolete and can be removed: You see that by checking the call hierarchy of this method. It's used by `grantLeadership` and only called if in running state (and the leadership is granted).



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {

Review Comment:
   This would swallow a potential IOException in the `hasJobResultEntryAsync` call. The old implementation handled these kind of errors. PTAL



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {
+                                                            return null;
+                                                        }
+                                                        if (hasJobResult) {
+                                                            runIfValidLeader(
+                                                                    leaderSessionId,
+                                                                    () -> {
+                                                                        jobAlreadyDone(
+                                                                                leaderSessionId);
+                                                                        return CompletableFuture
+                                                                                .completedFuture(
+                                                                                        null);
+                                                                    },
+                                                                    "check completed job");
+                                                        } else {
+                                                            runIfValidLeader(
+                                                                    leaderSessionId,
+                                                                    () -> {
+                                                                        ThrowingRunnable.unchecked(
+                                                                                        () ->
+                                                                                                createNewJobMasterServiceProcess(
+                                                                                                        leaderSessionId))
+                                                                                .run();
+                                                                        return CompletableFuture
+                                                                                .completedFuture(
+                                                                                        null);
+                                                                    },
+                                                                    "create new job master service process");
+                                                        }
+                                                        return null;
+                                                    }));
+        } else {
+            LOG.trace(
+                    "Ignore leader action '{}' because the leadership runner is no longer the valid leader for {}.",
+                    "verify job scheduling status and create JobMasterServiceProcess",
+                    leaderSessionId);
+        }
         handleAsyncOperationError(sequentialOperation, "Could not start the job manager.");
     }
 
     @GuardedBy("lock")
     private void verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(UUID leaderSessionId)

Review Comment:
   That method is not used anymore.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(

Review Comment:
   That works. But you could also call the `handle` on the result of the `thenCompose` call. I hope that this would improve readability because we would not have so many indentations. Another way to fix these indentation/readability issues is to move the async code into its own private method with a meaningful name.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")

Review Comment:
   ```suggestion
                                                       "verify job result entry")
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -488,18 +523,21 @@ private boolean isRunning() {
         return state == State.RUNNING;
     }
 
-    private void runIfValidLeader(
-            UUID expectedLeaderId, Runnable action, String actionDescription) {
+    private <T> CompletableFuture<T> runIfValidLeader(

Review Comment:
   nit: This method should be called `supplyIfValidLeader`. But with your current implementation, a dedicated `runIfValidLeader` would be still helpful. Hint: SupplierAsync-based and Runnable-based helper methods can be translated to each other in the following way:
   ```
   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 runIfValidLeader(
               UUID expectedLeaderId, Runnable action, Runnable noLeaderFallback) {
           synchronized (lock) {
               if (isValidLeader(expectedLeaderId)) {
                   action.run();
               } else {
                   noLeaderFallback.run();
               }
           }
       }
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {
+                                                            return null;
+                                                        }
+                                                        if (hasJobResult) {
+                                                            runIfValidLeader(

Review Comment:
   I guess we could remove code redundancy here by moving the `runIfValidLeader` out of the if clause. Alternatively, you could move the content of the two if/else blocks into their own private method with meaningful names (e.g. `handleJobAlreadyDoneIfValidLeader(leaderSessionID)` and `createNewJobMasterServiceProcessIfValidLeader(leaderSessionID)`) to improve this part of the code.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283925443


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,21 +185,19 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {

Review Comment:
   Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,21 +185,19 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
         return fileSystem.exists(constructDirtyPath(jobId));
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283925581


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,21 +185,19 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
         return fileSystem.exists(constructDirtyPath(jobId));
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
         return fileSystem.exists(constructCleanPath(jobId));
     }
 
     @Override
     public Set<JobResult> getDirtyResultsInternal() throws IOException {
         createBasePathIfNeeded();
-
         final FileStatus[] statuses = fileSystem.listStatus(this.basePath);
-

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1297920162


##########
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:
   These codes have been 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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1542239945

   Hi @WencongLiu, sorry for the late response. I found time to look into you proposal now. The initial intention of FLINK-27204 was to have the async functionality being hidden in `FileSystemBasedJobResultStore`. The `FileSystemBasedJobResultStore` performs IO operations for most of the interface's methods which is considered blocking and should run in the `Dispatcher`'s main thread (as it does right now). To achieve that, we would have to migrate all `JobResultStore` methods to become asynchronous:
   | pre-FLINK-27204 JobResultStore | post-FLINK-27204 JobResultStore |
   |-----|-------|
   | void createDirtyResult(JobResultEntry) throws ... | CompletableFuture<Void> createDirtyResultAsync(JobResultEntry) |
   | void markResultAsClean(JobID) throws ... | CompletableFuture<Void> markResultAsCleanAsync(JobID) |
   | boolean hasJobResultEntry(JobID) throws ... | CompletableFuture<Boolean> hasJobResultEntryAsync(JobID) |
   | boolean hasDirtyJobResultEntry(JobID) throws ... | CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID) |
   | boolean hasCleanJobResultEntry(JobID) throws ... | CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID) |
   | Set<JobResult> getDirtyResults(JobID) throws ... | CompletableFuture<Set<JobResult>> getDirtyResultsAsync() |
   
   The `FileSystemBasedJobResultStore` would get an constructor parameter `ioExecutor` which then would be used to run the async calls. Your current proposal doesn't specify a executorService to run the CompletableFutures on. Additionally, we need to utilize the CompletableFutures whereever possible (instead of calling `.get()` rightaway). Calling `.get()` on the `CompletableFuture` makes the call blocking again (which is what we want to avoid).
   
   Does that sound reasonable to you? Let me know if you have more questions.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "flinkbot (via GitHub)" <gi...@apache.org>.
flinkbot commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1495442861

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c6554365ad45b02b93512f9741681fca7f3b4023",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c6554365ad45b02b93512f9741681fca7f3b4023",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c6554365ad45b02b93512f9741681fca7f3b4023 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264360249


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -77,7 +78,14 @@ public JobDispatcherLeaderProcessFactory createFactory(
         }
 
         final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore();
-        final Collection<JobResult> recoveredDirtyJobResults = getDirtyJobResults(jobResultStore);
+        Collection<JobResult> recoveredDirtyJobResults;

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1289991654


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -277,18 +277,11 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
 
     @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);
+            throws FlinkException, ExecutionException, InterruptedException {
+        if (jobResultStore.hasJobResultEntryAsync(getJobID()).get()) {

Review Comment:
   Ok, I looked into that one: It's actually not that tricky. We have to refactor the `thenRun` call context of this method (see [line 265](https://github.com/apache/flink/pull/22341/files#diff-3eb433f18b85c0f5329a4b312a219583189d777fe9bdd547f1114f4a22989f8bR265)) into a `thenCompose(..).handle(..)` chain. The async call of `hasJobResultEntryAsync` would be executed in the `thenCompose` step. The rest will be then handled in the subsequent `handle` call. WDYT? Do you need more guidance with that one or is this proposal helpful?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1302368927


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   Thanks for your detailed explanation! 😄 I've modified the `thenCompose` to  `thenComposeAsync`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   Thanks for your detailed explanation! 😄 I've modified the `thenCompose` to  `thenComposeAsync`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1301129820


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -338,32 +356,24 @@ private void confirmLeadership(
         FutureUtils.assertNoException(
                 leaderAddressFuture.thenAccept(
                         address -> {
-                            synchronized (lock) {
-                                if (isValidLeader(leaderSessionId)) {
-                                    LOG.debug("Confirm leadership {}.", leaderSessionId);
-                                    leaderElection.confirmLeadership(leaderSessionId, address);
-                                } else {
-                                    LOG.trace(
-                                            "Ignore confirming leadership because the leader {} is no longer valid.",
-                                            leaderSessionId);
-                                }
-                            }
+                            runIfValidLeader(
+                                    leaderSessionId,
+                                    () -> {
+                                        LOG.debug("Confirm leadership {}.", leaderSessionId);
+                                        leaderElection.confirmLeadership(leaderSessionId, address);
+                                    },
+                                    "confirming leadership");
                         }));
     }
 
     private void forwardResultFuture(
             UUID leaderSessionId, CompletableFuture<JobManagerRunnerResult> resultFuture) {
         resultFuture.whenComplete(
                 (jobManagerRunnerResult, throwable) -> {

Review Comment:
   ```suggestion
                   (jobManagerRunnerResult, throwable) -> 
   ```
   And while squashing the commits you can also remove the obsolete bracket here.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288206790


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        dirtyResultAsync.get();
         assertThat(emptyBaseDirectory).exists().isDirectory();

Review Comment:
   I have added inverted asserts in the tests in FileSystemJobResultStoreFileOperationsTest.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1298453225


##########
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:
   You missed `forwardIfValidLeader`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288107381


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -46,72 +47,97 @@ public interface JobResultStoreContractTest {
     @Test
     default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .join())
+                .isInstanceOf(CompletionException.class);

Review Comment:
   Sorry. Currently I also apply this to `testStoreDirtyEntryForAlreadyCleanedJobResultThrowsException`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295337708


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {

Review Comment:
   I've remove these records.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {

Review Comment:
   I've remove these codes.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1679905718

   Thanks for your patient review. @XComp  I have added a fixup commit. PTAL.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283923614


##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264354843


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,84 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            try {
+                                withWriteLock(() -> createDirtyResultInternal(jobResultEntry));

Review Comment:
   Done. I've refactored the `AbstractThreadsafeJobResultStore `, `EmbeddedJobResultStore `, and `FileSystemJobResultStore`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264361955


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,37 +188,70 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
-        return fileSystem.exists(constructDirtyPath(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryInternal(JobID jobId) {
+        CompletableFuture<Boolean> hasDirtyJobResultEntryFuture = new CompletableFuture<>();
+        ioExecutor.execute(
+                () -> {
+                    try {
+                        hasDirtyJobResultEntryFuture.complete(
+                                fileSystem.exists(constructDirtyPath(jobId)));
+                    } catch (IOException e) {
+                        hasDirtyJobResultEntryFuture.completeExceptionally(e);
+                    }
+                });
+        return hasDirtyJobResultEntryFuture;
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
-        return fileSystem.exists(constructCleanPath(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryInternal(JobID jobId) {
+        CompletableFuture<Boolean> hasCleanJobResultEntryFuture = new CompletableFuture<>();

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264360912


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,37 +188,70 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
-        return fileSystem.exists(constructDirtyPath(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryInternal(JobID jobId) {
+        CompletableFuture<Boolean> hasDirtyJobResultEntryFuture = new CompletableFuture<>();
+        ioExecutor.execute(
+                () -> {
+                    try {
+                        hasDirtyJobResultEntryFuture.complete(
+                                fileSystem.exists(constructDirtyPath(jobId)));
+                    } catch (IOException e) {
+                        hasDirtyJobResultEntryFuture.completeExceptionally(e);
+                    }
+                });
+        return hasDirtyJobResultEntryFuture;

Review Comment:
   Currently, asynchronous operation has been moved into`AbstractThreadsafeJobResultStore`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1270167580


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -103,8 +103,8 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
 
     private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Fixed. The method `getDirtyResultsAsync` is 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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1269643021


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -575,12 +576,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      */
     private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
         try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
+            return jobResultStore.hasJobResultEntryAsync(jobId).get();

Review Comment:
   again, calling `.get()` on a `CompletableFuture` just to create another `CompletableFuture` isn't how you would try to solve asynchronous calls. You should utilize the `CompletableFuture` methods to chain callbacks asynchronously.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1269645604


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -191,8 +191,8 @@ private Collection<JobResult> getDirtyJobResultsIfRunning() {
 
     private Collection<JobResult> getDirtyJobResults() {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   We could revert that change entirely because of the revert of `getDirtyResultsAsync`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1270162352


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,87 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {

Review Comment:
   Fixed. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283010789


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -580,17 +581,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      * Checks whether the given job has already been executed.
      *
      * @param jobId identifying the submitted job
-     * @return true if the job has already finished, either successfully or as a failure
+     * @return a successfully completed future with {@code true} if the job has already finished,
+     *     either successfully or as a failure
      * @throws FlinkException if the job scheduling status cannot be retrieved

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283948678


##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -116,58 +111,53 @@ public static TestingJobResultStore.Builder builder() {
     /** {@code Builder} for instantiating {@code TestingJobResultStore} instances. */
     public static class Builder {
 
-        private ThrowingConsumer<JobResultEntry, ? extends IOException> createDirtyResultConsumer =
-                ignored -> {};
-        private ThrowingConsumer<JobID, ? extends IOException> markResultAsCleanConsumer =
-                ignored -> {};
+        private Function<JobResultEntry, CompletableFuture<Void>> createDirtyResultConsumer =
+                jobResultEntry -> CompletableFuture.completedFuture(null);

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1206775003


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,69 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
+     * @return CompletableFuture with the completed state.
      * @throws IOException if the creation of the dirty result failed for IO reasons.
      * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
      *     attached that is already registered in this {@code JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
+     * @return CompletableFuture with the completed state.
      * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
      *     failed for IO reasons.
      * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
      *     store for the given {@code JobID}.

Review Comment:
   The `IOException` can be removed from the javaDoc now. It's going to be wrapped by the `CompletableFuture`. But we might want to mentioned the `NoSuchElementException` as part of the return value's documentation, still.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,69 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
+     * @return CompletableFuture with the completed state.
      * @throws IOException if the creation of the dirty result failed for IO reasons.
      * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
      *     attached that is already registered in this {@code JobResultStore}.

Review Comment:
   The IOException can be removed from the javaDoc now. It's going to be wrapped by the CompletableFuture. But we might want to mentioned the `IllegalStateException` as part of the return value's documentation, still.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,84 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            try {
+                                withWriteLock(() -> createDirtyResultInternal(jobResultEntry));

Review Comment:
   You have to be careful here: The internal methods are now only triggering the actual logic in the `ioExecutor`. Therefore, only submitting the task is guarded under the lock. You're losing the synchronization of the IO tasks which will run in multiple threads of the ioExecutor for the `FileSystemJobResultStore` implementation.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1361,28 +1363,35 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
-                        return;
-                    }
-                    writeFuture.complete(null);
-                });
+        jobResultStore
+                .hasCleanJobResultEntryAsync(jobId)
+                .handleAsync(

Review Comment:
   That's just a thought without having proven that, actuall, but: Would the code become more readible if we utilize the `CompletableFuture` methods (e.g. `thenCompose`)? :thinking: 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -575,12 +576,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      */
     private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
         try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
+            return jobResultStore.hasJobResultEntryAsync(jobId).get();

Review Comment:
   We're not gaining much here because the call is still blocking (essentially, changing the return value of `isInGloballyTerminalState` from `boolean` to CompletableFuture<Boolean>`): We would have to propagate the CompletableFuture down the call hierarchy to make use of it. 



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -44,106 +45,140 @@ public interface JobResultStoreContractTest {
     JobResultStore createJobResultStore() throws IOException;
 
     @Test
-    default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
+    default void testStoreJobResultsWithDuplicateIDsThrowsException()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .get())
+                .hasCauseInstanceOf(RuntimeException.class);
     }
 
     @Test
-    default void testStoreDirtyEntryForAlreadyCleanedJobResultThrowsException() throws IOException {
+    default void testStoreDirtyEntryForAlreadyCleanedJobResultThrowsException()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY))
-                .isInstanceOf(IllegalStateException.class);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get();
+        assertThatThrownBy(
+                        () -> jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get())
+                .hasCauseInstanceOf(RuntimeException.class);
     }
 
     @Test
-    default void testCleaningDuplicateEntryThrowsNoException() throws IOException {
+    default void testCleaningDuplicateEntryThrowsNoException()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get();
         assertThatNoException()
                 .isThrownBy(
-                        () -> jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()));
+                        () ->
+                                jobResultStore
+                                        .markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                        .get());
     }
 
     @Test
     default void testCleaningNonExistentEntryThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
         assertThatThrownBy(
-                        () -> jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()))
-                .isInstanceOf(NoSuchElementException.class);
+                        () ->
+                                jobResultStore
+                                        .markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                        .get())
+                .hasCauseInstanceOf(NoSuchElementException.class);
     }
 
     @Test
-    default void testHasJobResultEntryWithDirtyEntry() throws IOException {
+    default void testHasJobResultEntryWithDirtyEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        assertThat(jobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        assertThat(
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
                 .isTrue();
-        assertThat(jobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        assertThat(
+                        jobResultStore
+                                .hasCleanJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
                 .isFalse();
-        assertThat(jobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())).isTrue();
+        assertThat(jobResultStore.hasJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get())
+                .isTrue();
     }
 
     @Test
-    default void testHasJobResultEntryWithCleanEntry() throws IOException {
+    default void testHasJobResultEntryWithCleanEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-        assertThat(jobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get();
+        assertThat(
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
                 .isFalse();
-        assertThat(jobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId()))
+        assertThat(
+                        jobResultStore
+                                .hasCleanJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                .get())
+                .isTrue();
+        assertThat(jobResultStore.hasJobResultEntryAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).get())
                 .isTrue();
-        assertThat(jobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())).isTrue();
     }
 
     @Test
-    default void testHasJobResultEntryWithEmptyStore() throws IOException {
+    default void testHasJobResultEntryWithEmptyStore()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
         JobID jobId = new JobID();
-        assertThat(jobResultStore.hasDirtyJobResultEntry(jobId)).isFalse();
-        assertThat(jobResultStore.hasCleanJobResultEntry(jobId)).isFalse();
-        assertThat(jobResultStore.hasJobResultEntry(jobId)).isFalse();
+        assertThat(jobResultStore.hasDirtyJobResultEntryAsync(jobId).get()).isFalse();
+        assertThat(jobResultStore.hasCleanJobResultEntryAsync(jobId).get()).isFalse();
+        assertThat(jobResultStore.hasJobResultEntryAsync(jobId).get()).isFalse();
     }
 
     @Test
-    default void testGetDirtyResultsWithNoEntry() throws IOException {
+    default void testGetDirtyResultsWithNoEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        assertThat(jobResultStore.getDirtyResults()).isEmpty();
+        assertThat(jobResultStore.getDirtyResultsAsync().get()).isEmpty();
     }
 
     @Test
-    default void testGetDirtyResultsWithDirtyEntry() throws IOException {
+    default void testGetDirtyResultsWithDirtyEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();
         assertThat(
-                        jobResultStore.getDirtyResults().stream()
+                        jobResultStore.getDirtyResultsAsync().get().stream()
                                 .map(JobResult::getJobId)
                                 .collect(Collectors.toList()))
                 .singleElement()
                 .isEqualTo(DUMMY_JOB_RESULT_ENTRY.getJobId());
     }
 
     @Test
-    default void testGetDirtyResultsWithDirtyAndCleanEntry() throws IOException {
+    default void testGetDirtyResultsWithDirtyAndCleanEntry()
+            throws IOException, ExecutionException, InterruptedException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).get();

Review Comment:
   one tip: Using `join()` instead of `get()` reduces the diff for test methods because you wouldn't have to specifying `ExecutionException` and `InterruptedException` in the test method's signature.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -191,8 +191,8 @@ private Collection<JobResult> getDirtyJobResultsIfRunning() {
 
     private Collection<JobResult> getDirtyJobResults() {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Here, it's more obvious. We're making this call blocking again just to generate a CompletableFuture out of it again in line 124. Instead, we could propagate the `CompletableFuture`. WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,84 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            try {
+                                withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+                            } catch (IOException e) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            return null;
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);

Review Comment:
   Removing the debug log makes the `LOG` field unused.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -103,8 +103,8 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
 
     private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Same as in the `Dispatcher` code: It might be better to propagate the `CompletableFuture` for readability purposes. The calling code should decide whether it wants to wait for the async operation or not. It doesn't make that much of a difference in that case because we want to block and wait for the completion in that specific case. Therefore, feel free to counter-argue. :-)



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -575,12 +576,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      */
     private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
         try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
+            return jobResultStore.hasJobResultEntryAsync(jobId).get();

Review Comment:
   The `CompletableFuture` would then be utilized in `Dispatcher.submitJob`. FYI: There's another issue with the `Dispatcher` that is related and might be worth being resolved beforehand: FLINK-32098



##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -80,33 +82,65 @@ private TestingJobResultStore(
     }
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        createDirtyResultConsumer.accept(jobResultEntry);
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        try {
+            createDirtyResultConsumer.accept(jobResultEntry);
+        } catch (IOException e) {
+            CompletableFuture<Void> future = new CompletableFuture<>();
+            future.completeExceptionally(e);
+            return future;
+        }
+        return CompletableFuture.completedFuture(null);
     }
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException {
-        markResultAsCleanConsumer.accept(jobId);
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        try {
+            markResultAsCleanConsumer.accept(jobId);
+        } catch (IOException e) {

Review Comment:
   You're not following the contract here, I guess. :thinking: Usually for this `Testing*` implementations, we would change the callback type (e.g. a `ThrowingConsumer<T>` becomes a `Function<T, CompletableFuture<Void>`). That enables us to have the entire logic being specified in the test method.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1257,10 +1259,10 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
 
     private void markJobAsClean(JobID jobId) {
         try {
-            jobResultStore.markResultAsClean(jobId);
+            jobResultStore.markResultAsCleanAsync(jobId).get();

Review Comment:
   Here, it's not really an issue because we're calling it asynchronously in [Dispatcher:1249](https://github.com/apache/flink/blob/3547aaacd8aa2e72fe227c9ac97389e803bdb460/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java#L1249), anyway. But returning a future here would make the `markJobAsClean` method would make the method signature align closer to its purpose (and maybe even adding the `Async` suffix here?).



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264359072


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created
+     *     successfully, otherwise will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry);

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264360150


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########


Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264364274


##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -548,7 +557,10 @@ public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exce
                 TestingJobResultStore.builder()
                         .withCreateDirtyResultConsumer(
                                 jobResult -> {
-                                    throw new IOException("Expected IOException.");
+                                    CompletableFuture<Boolean> future = new CompletableFuture<>();
+                                    future.completeExceptionally(
+                                            new IOException("Expected IOException."));
+                                    return future;

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1500819380

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1498895544

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1207795244


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,84 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            try {
+                                withWriteLock(() -> createDirtyResultInternal(jobResultEntry));

Review Comment:
   This feedback is very clear. Modifying the interface of JobStore makes it difficult for `AbstractThreadsafeJobResultStore` to ensure thread safety when performing IO operations on related implementation classes `EmbeddedJobResultStore` and `FileSystemJobResultStore`. I think another way is that we remove the `AbstractThreadsafeJobResultStore`, and let these two implementations to promise thread safety in themselves. 😄



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1299741764


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########


Review Comment:
   The `JobMasterServiceLeadershipRunner` change in 042459fa should be moved into FLINK-27204 commit (042459fa).



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -515,7 +516,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
 
         try {
-            if (isInGloballyTerminalState(jobID)) {
+            if (isInGloballyTerminalState(jobID).get()) {

Review Comment:
   oh, I missed that one. Sorry. Can we switch to async processing here as well?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1652958163

   The CI has passed and pull request is ready for review now.😊


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283926122


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean}
+     *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise a successfully
+     *     completed future with {@code false}.
      */
-    default boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId);
+    default CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return hasDirtyJobResultEntryAsync(jobId)
+                .thenCombine(
+                        hasCleanJobResultEntryAsync(jobId),
+                        (result1, result2) -> result1 || result2);
     }
 
     /**
-     * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code dirty} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true}, if a {@code dirty} entry exists
+     *     for the given {@code JobID}; otherwise a successfully completed future with {@code
+     *     false}.
      */
-    boolean hasDirtyJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains a {@code clean} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code clean} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code clean} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true}, if a {@code clean} entry exists
+     *     for the given {@code JobID}; otherwise a successfully completed future with {@code
+     *     false}.
      */
-    boolean hasCleanJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId);
 
     /**
-     * Get the persisted {@link JobResult} instances that are marked as {@code dirty}. This is
+     * Returns persisted {@link JobResult} instances that are marked as {@code dirty}. This is
      * useful for recovery of finalization steps.
      *
-     * @return A set of dirty {@code JobResults} from the store.
-     * @throws IOException if collecting the set of dirty results failed for IO reasons.
+     * @return a set of dirty {@code JobResults} from the store.

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283926936


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1284129028


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -516,7 +517,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
 
         try {
             if (isDuplicateJob(jobGraph.getJobID())) {
-                if (isInGloballyTerminalState(jobGraph.getJobID())) {
+                if (isInGloballyTerminalState(jobGraph.getJobID()).get()) {

Review Comment:
   Fixed. I've fixed it with the async style.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295337708


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {

Review Comment:
   I've removed these codes.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295631119


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {

Review Comment:
   The `if (hasJobResult == null) {` block becomes obsolete here (because there shouldn't be a situation where no result is passed if no error happened in upstream async calls for `handle(BiFunction)`. You could add a `Precondition.checkNotNull` if you like. But the subsequent if call would cause a `NullPointerException` anyway which makes the precondition obsolete again.
   
   (... and a side-remark here: we would at least add logs rather than only returning `null` in case if the `if (hasJobResult == null) {` case.)



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1298451802


##########
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:
   You missed `forwardIfValidLeader`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264360081


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created
+     *     successfully, otherwise will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
+     * @return CompletableFuture is the future with the completed state, which will throw {@link

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288094128


##########
flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java:
##########
@@ -381,9 +381,12 @@ public static void rethrowException(Throwable t) throws Exception {
      * @param e exception to throw if not null.
      * @throws Exception
      */
-    public static void tryRethrowException(@Nullable Exception e) throws Exception {
+    public static void tryRethrowException(@Nullable Throwable e) throws Exception {

Review Comment:
   The change of `ExceptionUtils` is reverted.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288102574


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########


Review Comment:
   The commit message and the content of commit has been modified.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288206790


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        dirtyResultAsync.get();
         assertThat(emptyBaseDirectory).exists().isDirectory();

Review Comment:
   I have added inverted asserts in the tests of FileSystemJobResultStoreFileOperationsTest.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288238489


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        dirtyResultAsync.get();

Review Comment:
   I've modified all `future.get()` to `FlinkAssertions.assertThatFuture`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283928788


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean}
+     *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise a successfully
+     *     completed future with {@code false}.
      */
-    default boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId);
+    default CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return hasDirtyJobResultEntryAsync(jobId)
+                .thenCombine(
+                        hasCleanJobResultEntryAsync(jobId),
+                        (result1, result2) -> result1 || result2);
     }
 
     /**
-     * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code dirty} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true}, if a {@code dirty} entry exists
+     *     for the given {@code JobID}; otherwise a successfully completed future with {@code

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264364321


##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -579,10 +591,17 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws
         final CompletableFuture<JobResultEntry> dirtyJobFuture = new CompletableFuture<>();
         final JobResultStore jobResultStore =
                 TestingJobResultStore.builder()
-                        .withCreateDirtyResultConsumer(dirtyJobFuture::complete)
+                        .withCreateDirtyResultConsumer(
+                                jobResultEntry -> {
+                                    dirtyJobFuture.complete(jobResultEntry);
+                                    return CompletableFuture.completedFuture(true);
+                                })
                         .withMarkResultAsCleanConsumer(
                                 jobId -> {
-                                    throw new IOException("Expected IOException.");
+                                    CompletableFuture<Void> result = new CompletableFuture<>();
+                                    result.completeExceptionally(
+                                            new IOException("Expected IOException."));
+                                    return result;

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264365814


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreContractTest.java:
##########
@@ -35,6 +35,6 @@ public class FileSystemJobResultStoreContractTest implements JobResultStoreContr
     @Override
     public JobResultStore createJobResultStore() throws IOException {
         Path path = new Path(temporaryFolder.toURI());
-        return new FileSystemJobResultStore(path.getFileSystem(), path, false);
+        return new FileSystemJobResultStore(path.getFileSystem(), path, false, Runnable::run);

Review Comment:
   Fixed.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -54,11 +58,19 @@ public class FileSystemJobResultStoreFileOperationsTest {
 
     private Path basePath;
 
+    private ExecutorService ioExecutor;
+
     @BeforeEach
     public void setupTest() throws IOException {
         basePath = new Path(temporaryFolder.toURI());
+        ioExecutor = Executors.newSingleThreadExecutor();

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264360063


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288185269


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -54,11 +57,15 @@ public class FileSystemJobResultStoreFileOperationsTest {
 
     private Path basePath;
 
+    final ManuallyTriggeredScheduledExecutor manuallyTriggeredExecutor =

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288110488


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -561,21 +563,28 @@ public CompletableFuture<Acknowledge> submitFailedJob(
         return archiveExecutionGraphToHistoryServer(executionGraphInfo);
     }
 
+    /**
+     * Checks whether the given job has already been submitted, executed, or awaiting termination.
+     *
+     * @param jobId identifying the submitted job
+     * @return true if the job has already been submitted (is running) or has been executed
+     * @throws Exception if the job scheduling status cannot be retrieved
+     */
+    private boolean isDuplicateJob(JobID jobId) throws Exception {
+        return isInGloballyTerminalState(jobId).get()
+                || jobManagerRunnerRegistry.isRegistered(jobId)
+                || submittedAndWaitingTerminationJobIDs.contains(jobId);
+    }
+

Review Comment:
   These codes is removed now.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1281945188


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -516,7 +517,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
 
         try {
             if (isDuplicateJob(jobGraph.getJobID())) {
-                if (isInGloballyTerminalState(jobGraph.getJobID())) {
+                if (isInGloballyTerminalState(jobGraph.getJobID()).get()) {

Review Comment:
   Reminder: That's going to create conflicts when rebasing this branch to master the next time.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -77,7 +77,14 @@ public JobDispatcherLeaderProcessFactory createFactory(
         }
 
         final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore();
-        final Collection<JobResult> recoveredDirtyJobResults = getDirtyJobResults(jobResultStore);
+        final Collection<JobResult> recoveredDirtyJobResults;

Review Comment:
   This class/file doesn't need to be changed anymore after moving back to `JobResultStore#getDirtyResult`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -516,7 +517,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
 
         try {
             if (isDuplicateJob(jobGraph.getJobID())) {
-                if (isInGloballyTerminalState(jobGraph.getJobID())) {
+                if (isInGloballyTerminalState(jobGraph.getJobID()).get()) {

Review Comment:
   Additionally, we shouldn't call `.get()` here where the error is the caught in a `catch` class returning a separate `CompletableFuture` again. Can't we use the Future methods here? Or is that harder to read? :thinking: 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -580,17 +581,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      * Checks whether the given job has already been executed.
      *
      * @param jobId identifying the submitted job
-     * @return true if the job has already finished, either successfully or as a failure
+     * @return a successfully completed future with {@code true} if the job has already finished,
+     *     either successfully or as a failure
      * @throws FlinkException if the job scheduling status cannot be retrieved

Review Comment:
   ```suggestion
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1390,29 +1392,71 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
+        CompletableFuture<Boolean> shouldCheckDirtyJobResult =
+                jobResultStore
+                        .hasCleanJobResultEntryAsync(jobId)
+                        .handleAsync(
+                                (hasCleanJobResultEntry, throwable) -> {
+                                    if (throwable != null) {
+                                        writeFuture.completeExceptionally(throwable);
+                                        return false;
+                                    } else {
+                                        if (hasCleanJobResultEntry) {
+                                            log.warn(
+                                                    "Job {} is already marked as clean but "
+                                                            + "clean up was triggered again.",
+                                                    jobId);
+                                            writeFuture.complete(null);
+                                            return false;
+                                        } else {
+                                            return true;
+                                        }
+                                    }
+                                });
+        shouldCheckDirtyJobResult.whenCompleteAsync(
+                (shouldCheck, throwable1) -> {
+                    if (throwable1 != null) {
+                        writeFuture.completeExceptionally(throwable1);
                         return;
                     }
-                    writeFuture.complete(null);
+                    if (shouldCheck) {
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(jobId)
+                                .whenCompleteAsync(
+                                        (hasDirtyJobResultEntry, throwable2) -> {
+                                            if (throwable2 != null) {
+                                                writeFuture.completeExceptionally(throwable2);
+                                                return;
+                                            }
+                                            if (!hasDirtyJobResultEntry) {
+                                                jobResultStore
+                                                        .createDirtyResultAsync(
+                                                                new JobResultEntry(
+                                                                        JobResult.createFrom(
+                                                                                archivedExecutionGraph)))
+                                                        .whenCompleteAsync(
+                                                                (unused, throwable3) -> {
+                                                                    if (throwable3 != null) {
+                                                                        writeFuture
+                                                                                .completeExceptionally(
+                                                                                        throwable3);
+                                                                        return;
+                                                                    }
+                                                                    log.info(
+                                                                            "Job {} has been registered "
+                                                                                    + "for cleanup in "
+                                                                                    + "the JobResultStore "
+                                                                                    + "after reaching a "
+                                                                                    + "terminal state.",
+                                                                            jobId);
+                                                                    writeFuture.complete(null);
+                                                                });
+                                            } else {
+                                                writeFuture.complete(null);
+                                            }
+                                        });
+                    }

Review Comment:
   This code block can be simplified (we don't need an extra `writeFuture` anymore) in the following way:
   ```
   return jobResultStore
                   .hasCleanJobResultEntryAsync(jobId)
                   .thenCompose(
                           hasCleanJobResultEntry -> {
                               if (hasCleanJobResultEntry) {
                                   log.warn(
                                           "Job {} is already marked as clean but clean up was triggered again.",
                                           jobId);
                                   return FutureUtils.completedVoidFuture();
                               } else {
                                   return jobResultStore
                                           .hasDirtyJobResultEntryAsync(jobId)
                                           .thenCompose(
                                                   hasDirtyJobResultEntry -> {
                                                       if (hasDirtyJobResultEntry) {
                                                           return FutureUtils.completedVoidFuture();
                                                       }
                                                       return jobResultStore.createDirtyResultAsync(
                                                               new JobResultEntry(
                                                                       JobResult.createFrom(
                                                                               archivedExecutionGraph)));
                                                   });
                               }
                           })
                   .handleAsync(
                           (ignored, error) -> {
                               if (error != null) {
                                   fatalErrorHandler.onFatalError(
                                           new FlinkException(
                                                   String.format(
                                                           "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.",
                                                           executionGraphInfo.getJobId()),
                                                   error));
                               }
                               return CleanupJobState.globalCleanup(terminalJobStatus);
                           },
                           getMainThreadExecutor());
   ```
   WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code

Review Comment:
   It's not the method that will throw the exception. The future will complete exceptionally.



##########
flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java:
##########
@@ -381,9 +381,12 @@ public static void rethrowException(Throwable t) throws Exception {
      * @param e exception to throw if not null.
      * @throws Exception
      */
-    public static void tryRethrowException(@Nullable Exception e) throws Exception {
+    public static void tryRethrowException(@Nullable Throwable e) throws Exception {

Review Comment:
   ```suggestion
       public static void tryRethrow(@Nullable Throwable e) throws Throwable {
   ```
   I'm wondering whether we should refactor this properly in a separate hotfix commit. WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean}
+     *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise a successfully
+     *     completed future with {@code false}.
      */
-    default boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId);
+    default CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return hasDirtyJobResultEntryAsync(jobId)
+                .thenCombine(
+                        hasCleanJobResultEntryAsync(jobId),
+                        (result1, result2) -> result1 || result2);
     }
 
     /**
-     * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code dirty} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true}, if a {@code dirty} entry exists
+     *     for the given {@code JobID}; otherwise a successfully completed future with {@code

Review Comment:
   ```suggestion
        *     for the given {@code JobID}; otherwise {@code
   ```
   nit: no need to add this - it should be understood



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########
@@ -57,17 +61,17 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) {
         return dirtyJobResults.containsKey(jobId);
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasCleanJobResultEntryInternal(JobID jobId) {

Review Comment:
   ```suggestion
       public boolean hasCleanJobResultEntryInternal(JobID jobId) {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -277,18 +277,11 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
 
     @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);
+            throws FlinkException, ExecutionException, InterruptedException {
+        if (jobResultStore.hasJobResultEntryAsync(getJobID()).get()) {

Review Comment:
   This one, I feel like addressing in a separate review because it requires some deeper changes. But we could leverage the async nature here as well.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.trigger();

Review Comment:
   hint: With my proposal of the `AbstractThreadsafeJobResultStore` these kind of tests will start to timeout because we're having two separate async operations being performed (checking that no entry exist and the actual operation of creating the file). Therefore, if you go with my proposal (or something similar), you would have to change from `trigger()` to `triggerAll()` here and in other test methods.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -116,58 +111,53 @@ public static TestingJobResultStore.Builder builder() {
     /** {@code Builder} for instantiating {@code TestingJobResultStore} instances. */
     public static class Builder {
 
-        private ThrowingConsumer<JobResultEntry, ? extends IOException> createDirtyResultConsumer =
-                ignored -> {};
-        private ThrowingConsumer<JobID, ? extends IOException> markResultAsCleanConsumer =
-                ignored -> {};
+        private Function<JobResultEntry, CompletableFuture<Void>> createDirtyResultConsumer =
+                jobResultEntry -> CompletableFuture.completedFuture(null);

Review Comment:
   ```suggestion
                   jobResultEntry -> FutureUtils.completedVoidFuture();
   ```
   That's a really nitty thing. Feel free to ignore it. I just wanted to point out that `FutureUtils` has a utility method for that (that would also apply to some of the other fields if you want to change that).



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));

Review Comment:
   That is correct. But we don't need the `RuntimeException` anymore. We can just use `FutureUtils.completeExceptionally(e)` with the `InterruptedException` here



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,21 +185,19 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
         return fileSystem.exists(constructDirtyPath(jobId));
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
         return fileSystem.exists(constructCleanPath(jobId));
     }
 
     @Override
     public Set<JobResult> getDirtyResultsInternal() throws IOException {
         createBasePathIfNeeded();
-
         final FileStatus[] statuses = fileSystem.listStatus(this.basePath);
-

Review Comment:
   Is there a reason why you removed these lines?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,21 +185,19 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
         return fileSystem.exists(constructDirtyPath(jobId));
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {

Review Comment:
   ```suggestion
       public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
   ```
   This change is not necessary



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,61 +47,126 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
+
+    public AbstractThreadsafeJobResultStore(Executor ioExecutor) {

Review Comment:
   ```suggestion
       protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########


Review Comment:
   Maybe it helps to compare your approach with the proposal above to get a better understanding of the `CompletableFuture` usage. Feel free to ask questions about it. And don't hesitate to question my approach! See it as a discussion item. :-)



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########


Review Comment:
   I'm gonna provide a general proposal for this class. It already looks good. I just think that we could leverage the CompletableFuture utility methods in a more efficient way. PTAL:
   ```
   package org.apache.flink.runtime.highavailability;
   
   import org.apache.flink.api.common.JobID;
   import org.apache.flink.runtime.jobmaster.JobResult;
   import org.apache.flink.util.Preconditions;
   import org.apache.flink.util.concurrent.FutureUtils;
   import org.apache.flink.util.function.SupplierWithException;
   import org.apache.flink.util.function.ThrowingRunnable;
   
   import org.slf4j.Logger;
   import org.slf4j.LoggerFactory;
   
   import javax.annotation.concurrent.GuardedBy;
   
   import java.io.IOException;
   import java.util.NoSuchElementException;
   import java.util.Set;
   import java.util.concurrent.CompletableFuture;
   import java.util.concurrent.Executor;
   import java.util.concurrent.locks.ReadWriteLock;
   import java.util.concurrent.locks.ReentrantReadWriteLock;
   
   /** An abstract class for threadsafe implementations of the {@link JobResultStore}. */
   public abstract class AbstractThreadsafeJobResultStore implements JobResultStore {
   
       private static final Logger LOG =
               LoggerFactory.getLogger(AbstractThreadsafeJobResultStore.class);
   
       private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
   
       private final Executor ioExecutor;
   
       protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
           this.ioExecutor = ioExecutor;
       }
   
       @Override
       public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
           return hasJobResultEntryAsync(jobResultEntry.getJobId())
                   .thenAccept(
                           hasJobResultEntry ->
                                   Preconditions.checkState(
                                           !hasJobResultEntry,
                                           "Job result store already contains an entry for job %s",
                                           jobResultEntry.getJobId()))
                   .thenCompose(
                           ignoredVoid ->
                                   withWriteLockAsync(
                                           () -> createDirtyResultInternal(jobResultEntry)));
       }
   
       @GuardedBy("readWriteLock")
       protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
               throws IOException;
   
       @Override
       public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
           return hasCleanJobResultEntryAsync(jobId)
                   .thenCompose(
                           hasCleanJobResultEntry -> {
                               if (hasCleanJobResultEntry) {
                                   LOG.debug(
                                           "The job {} is already marked as clean. No action required.",
                                           jobId);
                                   return FutureUtils.completedVoidFuture();
                               }
   
                               return withWriteLockAsync(() -> markResultAsCleanInternal(jobId));
                           });
       }
   
       @GuardedBy("readWriteLock")
       protected abstract void markResultAsCleanInternal(JobID jobId)
               throws IOException, NoSuchElementException;
   
       @Override
       public CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
           return withReadLockAsync(
                   () ->
                           hasDirtyJobResultEntryInternal(jobId)
                                   || hasCleanJobResultEntryInternal(jobId));
       }
   
       @Override
       public CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId) {
           return withReadLockAsync(() -> hasDirtyJobResultEntryInternal(jobId));
       }
   
       @GuardedBy("readWriteLock")
       protected abstract Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException;
   
       @Override
       public CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId) {
           return withReadLockAsync(() -> hasCleanJobResultEntryInternal(jobId));
       }
   
       @GuardedBy("readWriteLock")
       protected abstract Boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException;
   
       @Override
       public Set<JobResult> getDirtyResults() throws IOException {
           return getDirtyResultsInternal();
       }
   
       @GuardedBy("readWriteLock")
       protected abstract Set<JobResult> getDirtyResultsInternal() throws IOException;
   
       private CompletableFuture<Void> withWriteLockAsync(ThrowingRunnable<IOException> runnable) {
           return FutureUtils.runAsync(() -> withWriteLock(runnable), ioExecutor);
       }
   
       private void withWriteLock(ThrowingRunnable<IOException> runnable) throws IOException {
           readWriteLock.writeLock().lock();
           try {
               runnable.run();
           } finally {
               readWriteLock.writeLock().unlock();
           }
       }
   
       private <T> CompletableFuture<T> withReadLockAsync(
               SupplierWithException<T, IOException> runnable) {
           return FutureUtils.supplyAsync(() -> withReadLock(runnable), ioExecutor);
       }
   
       private <T> T withReadLock(SupplierWithException<T, IOException> supplier) throws IOException {
           readWriteLock.readLock().lock();
           try {
               return supplier.get();
           } finally {
               readWriteLock.readLock().unlock();
           }
       }
   }
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean}
+     *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise a successfully
+     *     completed future with {@code false}.
      */
-    default boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId);
+    default CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return hasDirtyJobResultEntryAsync(jobId)
+                .thenCombine(
+                        hasCleanJobResultEntryAsync(jobId),
+                        (result1, result2) -> result1 || result2);
     }
 
     /**
-     * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code dirty} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true}, if a {@code dirty} entry exists
+     *     for the given {@code JobID}; otherwise a successfully completed future with {@code
+     *     false}.
      */
-    boolean hasDirtyJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains a {@code clean} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code clean} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code clean} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true}, if a {@code clean} entry exists
+     *     for the given {@code JobID}; otherwise a successfully completed future with {@code
+     *     false}.
      */
-    boolean hasCleanJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId);
 
     /**
-     * Get the persisted {@link JobResult} instances that are marked as {@code dirty}. This is
+     * Returns persisted {@link JobResult} instances that are marked as {@code dirty}. This is
      * useful for recovery of finalization steps.
      *
-     * @return A set of dirty {@code JobResults} from the store.
-     * @throws IOException if collecting the set of dirty results failed for IO reasons.
+     * @return a set of dirty {@code JobResults} from the store.

Review Comment:
   The JavaDoc should be reverted as well.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1273,7 +1270,22 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
         if (cleanupJobState.isGlobalCleanup()) {
             return globalResourceCleaner
                     .cleanupAsync(jobId)
-                    .thenRunAsync(() -> markJobAsClean(jobId), ioExecutor)
+                    .thenCompose(unused -> jobResultStore.markResultAsCleanAsync(jobId))
+                    .handle(
+                            (BiFunction<Void, Throwable, Void>)
+                                    (unused, e) -> {

Review Comment:
   ```suggestion
                       .thenCompose(unusedVoid -> jobResultStore.markResultAsCleanAsync(jobId))
                       .handle(
                                       (unusedVoid, error) -> {
   ```
   The cast is not necessary. Instead, you could use meaningful parameter names that explain the purpose of each parameter.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1390,29 +1392,71 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
+        CompletableFuture<Boolean> shouldCheckDirtyJobResult =
+                jobResultStore
+                        .hasCleanJobResultEntryAsync(jobId)
+                        .handleAsync(
+                                (hasCleanJobResultEntry, throwable) -> {
+                                    if (throwable != null) {
+                                        writeFuture.completeExceptionally(throwable);
+                                        return false;
+                                    } else {
+                                        if (hasCleanJobResultEntry) {
+                                            log.warn(
+                                                    "Job {} is already marked as clean but "
+                                                            + "clean up was triggered again.",
+                                                    jobId);
+                                            writeFuture.complete(null);
+                                            return false;
+                                        } else {
+                                            return true;
+                                        }
+                                    }
+                                });
+        shouldCheckDirtyJobResult.whenCompleteAsync(
+                (shouldCheck, throwable1) -> {
+                    if (throwable1 != null) {
+                        writeFuture.completeExceptionally(throwable1);
                         return;
                     }
-                    writeFuture.complete(null);
+                    if (shouldCheck) {
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(jobId)
+                                .whenCompleteAsync(
+                                        (hasDirtyJobResultEntry, throwable2) -> {
+                                            if (throwable2 != null) {
+                                                writeFuture.completeExceptionally(throwable2);
+                                                return;
+                                            }
+                                            if (!hasDirtyJobResultEntry) {
+                                                jobResultStore
+                                                        .createDirtyResultAsync(
+                                                                new JobResultEntry(
+                                                                        JobResult.createFrom(
+                                                                                archivedExecutionGraph)))
+                                                        .whenCompleteAsync(
+                                                                (unused, throwable3) -> {
+                                                                    if (throwable3 != null) {
+                                                                        writeFuture
+                                                                                .completeExceptionally(
+                                                                                        throwable3);
+                                                                        return;
+                                                                    }
+                                                                    log.info(
+                                                                            "Job {} has been registered "
+                                                                                    + "for cleanup in "
+                                                                                    + "the JobResultStore "
+                                                                                    + "after reaching a "
+                                                                                    + "terminal state.",
+                                                                            jobId);
+                                                                    writeFuture.complete(null);
+                                                                });
+                                            } else {
+                                                writeFuture.complete(null);
+                                            }
+                                        });
+                    }

Review Comment:
   This could be split up into meaningful private methods to handle the individual if branches:
   ```
       private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobResultStore(
               ExecutionGraphInfo executionGraphInfo) {
           final JobID jobId = executionGraphInfo.getJobId();
   
           final AccessExecutionGraph archivedExecutionGraph =
                   executionGraphInfo.getArchivedExecutionGraph();
   
           final JobStatus terminalJobStatus = archivedExecutionGraph.getState();
           Preconditions.checkArgument(
                   terminalJobStatus.isGloballyTerminalState(),
                   "Job %s is in state %s which is not globally terminal.",
                   jobId,
                   terminalJobStatus);
   
           return jobResultStore
                   .hasCleanJobResultEntryAsync(jobId)
                   .thenCompose(
                           hasCleanJobResultEntry ->
                                   createDirtyJobResultEntryIfMissingAsync(
                                           archivedExecutionGraph, hasCleanJobResultEntry))
                   .handleAsync(
                           (ignored, error) -> {
                               if (error != null) {
                                   fatalErrorHandler.onFatalError(
                                           new FlinkException(
                                                   String.format(
                                                           "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.",
                                                           executionGraphInfo.getJobId()),
                                                   error));
                               }
                               return CleanupJobState.globalCleanup(terminalJobStatus);
                           },
                           getMainThreadExecutor());
       }
   
       /**
        * Creates a dirty entry in the {@link #jobResultStore} if there's no entry at all for the given
        * {@code executionGraph} in the {@code JobResultStore}.
        *
        * @param executionGraph The {@link AccessExecutionGraph} for which the {@link JobResult} shall
        *     be persisted.
        * @param hasCleanJobResultEntry The decision the dirty entry check is based on.
        * @return {@code CompletableFuture} that completes as soon as the entry exists.
        */
       private CompletableFuture<Void> createDirtyJobResultEntryIfMissingAsync(
               AccessExecutionGraph executionGraph, boolean hasCleanJobResultEntry) {
           final JobID jobId = executionGraph.getJobID();
           if (hasCleanJobResultEntry) {
               log.warn("Job {} is already marked as clean but clean up was triggered again.", jobId);
               return FutureUtils.completedVoidFuture();
           } else {
               return jobResultStore
                       .hasDirtyJobResultEntryAsync(jobId)
                       .thenCompose(
                               hasDirtyJobResultEntry ->
                                       createDirtyJobResultEntryAsync(
                                               executionGraph, hasDirtyJobResultEntry));
           }
       }
   
       /**
        * Creates a dirty entry in the {@link #jobResultStore} based on the passed {@code
        * hasDirtyJobResultEntry} flag.
        *
        * @param executionGraph The {@link AccessExecutionGraph} that is used to generate the entry.
        * @param hasDirtyJobResultEntry The decision the entry creation is based on.
        * @return {@code CompletableFuture} that completes as soon as the entry exists.
        */
       private CompletableFuture<Void> createDirtyJobResultEntryAsync(
               AccessExecutionGraph executionGraph, boolean hasDirtyJobResultEntry) {
           if (hasDirtyJobResultEntry) {
               return FutureUtils.completedVoidFuture();
           }
   
           return jobResultStore.createDirtyResultAsync(
                   new JobResultEntry(JobResult.createFrom(executionGraph)));
       }
   ```
   
   Having the individual methods should help to understand the actual subtasks when reading the code. WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean}
+     *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise a successfully
+     *     completed future with {@code false}.

Review Comment:
   ```suggestion
        *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise {@code false}.
   ```
   nit: no need to add this - it should be understood



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java:
##########
@@ -181,21 +185,19 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {

Review Comment:
   ```suggestion
       public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
   ```
   This change is not necessary



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -580,17 +581,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      * Checks whether the given job has already been executed.
      *
      * @param jobId identifying the submitted job
-     * @return true if the job has already finished, either successfully or as a failure
+     * @return a successfully completed future with {@code true} if the job has already finished,
+     *     either successfully or as a failure
      * @throws FlinkException if the job scheduling status cannot be retrieved
      */
-    private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
-        try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
-        }
+    private CompletableFuture<Boolean> isInGloballyTerminalState(JobID jobId)

Review Comment:
   Here it makes sense to keep the method (even though it's also just calling a single other method) because `isInGloballyTerminalState` is more descriptive than `jobResultStore.hasJobResultEntryAsync` :+1: 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########
@@ -57,17 +61,17 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) {

Review Comment:
   ```suggestion
       public boolean hasDirtyJobResultEntryInternal(JobID jobId) {
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -579,10 +587,15 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws
         final CompletableFuture<JobResultEntry> dirtyJobFuture = new CompletableFuture<>();
         final JobResultStore jobResultStore =
                 TestingJobResultStore.builder()
-                        .withCreateDirtyResultConsumer(dirtyJobFuture::complete)
+                        .withCreateDirtyResultConsumer(
+                                jobResultEntry -> {
+                                    dirtyJobFuture.complete(jobResultEntry);
+                                    return FutureUtils.completedVoidFuture();
+                                })
                         .withMarkResultAsCleanConsumer(
                                 jobId -> {
-                                    throw new IOException("Expected IOException.");
+                                    return FutureUtils.completedExceptionally(

Review Comment:
   This can be shortened (the brackets are obsolete in a oneliner). Analogously to your change above.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.

Review Comment:
   We should remove the documentation for the `NoSuchElementException`. That should be still part of the contract and, therefore, mentioned in the JavaDoc. Could you add this as part of the returned future documentation as an additional sentence?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########


Review Comment:
   Removing the `IOException` from the method signatures in `EmbeddedJobResultStore` could happen in a separate hotfix commit.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -46,72 +47,97 @@ public interface JobResultStoreContractTest {
     @Test
     default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .join())
+                .isInstanceOf(CompletionException.class);
     }
 
     @Test
     default void testStoreDirtyEntryForAlreadyCleanedJobResultThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY))
-                .isInstanceOf(IllegalStateException.class);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).join();
+        assertThatThrownBy(
+                        () -> jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join())
+                .isInstanceOf(CompletionException.class);
     }
 
     @Test
     default void testCleaningDuplicateEntryThrowsNoException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
+        jobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId()).join();
         assertThatNoException()
                 .isThrownBy(
-                        () -> jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()));
+                        () ->
+                                jobResultStore
+                                        .markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                        .join());
     }
 
     @Test
     default void testCleaningNonExistentEntryThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
         assertThatThrownBy(
-                        () -> jobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()))
-                .isInstanceOf(NoSuchElementException.class);
+                        () ->
+                                jobResultStore
+                                        .markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId())
+                                        .join())
+                .hasCauseInstanceOf(NoSuchElementException.class);

Review Comment:
   Here you asserted the cause properly. Well done :+1: 



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -46,72 +47,97 @@ public interface JobResultStoreContractTest {
     @Test
     default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .join())
+                .isInstanceOf(CompletionException.class);

Review Comment:
   we should also validate the cause of 
   ```suggestion
                   .isInstanceOf(CompletionException.class)
                   .hasCauseInstanceOf(IllegalStateException.class);
   ```
   We should also assert for the cause of the `CompletionException` because that's the actual business logic. This statement applies also to the other methods in this test class. I'm not gonna mark all of them individually.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));

Review Comment:
   We still should call `Thread.currentThread().interrupt()` before returning the exceptionally completed future. Something like that:
   ```
                                                       try {
                                                           markAsDirtyLatch.await();
                                                       } catch (InterruptedException e) {
                                                           Thread.currentThread().interrupt();
                                                           return FutureUtils.completedExceptionally(e);
                                                       }
                                                       return FutureUtils.completedVoidFuture();
   ```
   No local variable `result` is needed here.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -116,58 +111,53 @@ public static TestingJobResultStore.Builder builder() {
     /** {@code Builder} for instantiating {@code TestingJobResultStore} instances. */
     public static class Builder {
 
-        private ThrowingConsumer<JobResultEntry, ? extends IOException> createDirtyResultConsumer =
-                ignored -> {};
-        private ThrowingConsumer<JobID, ? extends IOException> markResultAsCleanConsumer =
-                ignored -> {};
+        private Function<JobResultEntry, CompletableFuture<Void>> createDirtyResultConsumer =
+                jobResultEntry -> CompletableFuture.completedFuture(null);
+        private Function<JobID, CompletableFuture<Void>> markResultAsCleanConsumer =
+                jobID -> CompletableFuture.completedFuture(null);
 
-        private FunctionWithException<JobID, Boolean, ? extends IOException>
-                hasJobResultEntryFunction = ignored -> false;
-        private FunctionWithException<JobID, Boolean, ? extends IOException>
-                hasDirtyJobResultEntryFunction = ignored -> false;
-        private FunctionWithException<JobID, Boolean, ? extends IOException>
-                hasCleanJobResultEntryFunction = ignored -> false;
+        private Function<JobID, CompletableFuture<Boolean>> hasJobResultEntryFunction =
+                jobID -> CompletableFuture.completedFuture(false);
+        private Function<JobID, CompletableFuture<Boolean>> hasDirtyJobResultEntryFunction =
+                jobID -> CompletableFuture.completedFuture(false);
+        private Function<JobID, CompletableFuture<Boolean>> hasCleanJobResultEntryFunction =
+                jobID -> CompletableFuture.completedFuture(false);
 
-        private SupplierWithException<Set<JobResult>, ? extends IOException>
-                getDirtyResultsSupplier = Collections::emptySet;
+        private Supplier<Set<JobResult>> dirtyResultsSupplier = Collections::emptySet;
 
         public Builder withCreateDirtyResultConsumer(
-                ThrowingConsumer<JobResultEntry, ? extends IOException> createDirtyResultConsumer) {
+                Function<JobResultEntry, CompletableFuture<Void>> createDirtyResultConsumer) {
             this.createDirtyResultConsumer = createDirtyResultConsumer;
             return this;
         }
 
         public Builder withMarkResultAsCleanConsumer(
-                ThrowingConsumer<JobID, ? extends IOException> markResultAsCleanConsumer) {
+                Function<JobID, CompletableFuture<Void>> markResultAsCleanConsumer) {
             this.markResultAsCleanConsumer = markResultAsCleanConsumer;
             return this;
         }
 
         public Builder withHasJobResultEntryFunction(
-                FunctionWithException<JobID, Boolean, ? extends IOException>
-                        hasJobResultEntryFunction) {
+                Function<JobID, CompletableFuture<Boolean>> hasJobResultEntryFunction) {
             this.hasJobResultEntryFunction = hasJobResultEntryFunction;
             return this;
         }
 
         public Builder withHasDirtyJobResultEntryFunction(
-                FunctionWithException<JobID, Boolean, ? extends IOException>
-                        hasDirtyJobResultEntryFunction) {
+                Function<JobID, CompletableFuture<Boolean>> hasDirtyJobResultEntryFunction) {
             this.hasDirtyJobResultEntryFunction = hasDirtyJobResultEntryFunction;
             return this;
         }
 
         public Builder withHasCleanJobResultEntryFunction(
-                FunctionWithException<JobID, Boolean, ? extends IOException>
-                        hasCleanJobResultEntryFunction) {
+                Function<JobID, CompletableFuture<Boolean>> hasCleanJobResultEntryFunction) {
             this.hasCleanJobResultEntryFunction = hasCleanJobResultEntryFunction;
             return this;
         }
 
         public Builder withGetDirtyResultsSupplier(
-                SupplierWithException<Set<JobResult>, ? extends IOException>
-                        getDirtyResultsSupplier) {
-            this.getDirtyResultsSupplier = getDirtyResultsSupplier;
+                Supplier<Set<JobResult>> getDirtyResultsSupplier) {

Review Comment:
   I guess the `getDirtyResults` field doesn't need to change.



##########
flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java:
##########
@@ -381,9 +381,12 @@ public static void rethrowException(Throwable t) throws Exception {
      * @param e exception to throw if not null.
      * @throws Exception
      */
-    public static void tryRethrowException(@Nullable Exception e) throws Exception {
+    public static void tryRethrowException(@Nullable Throwable e) throws Exception {

Review Comment:
   This change might not be necessary with the proposed changes below. :thinking: 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283102037


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,61 +47,126 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
+
+    public AbstractThreadsafeJobResultStore(Executor ioExecutor) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1282979301


##########
flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java:
##########
@@ -381,9 +381,12 @@ public static void rethrowException(Throwable t) throws Exception {
      * @param e exception to throw if not null.
      * @throws Exception
      */
-    public static void tryRethrowException(@Nullable Exception e) throws Exception {
+    public static void tryRethrowException(@Nullable Throwable e) throws Exception {

Review Comment:
   If the `throws Exception` is modified to `throws Throwable`, many classes that use the tryRethrowException have to `throws Throwable` 🤔 I'd prefer to refactor this in the old way and keep it in a separate hotfix commit. WDYT?



##########
flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java:
##########
@@ -381,9 +381,12 @@ public static void rethrowException(Throwable t) throws Exception {
      * @param e exception to throw if not null.
      * @throws Exception
      */
-    public static void tryRethrowException(@Nullable Exception e) throws Exception {
+    public static void tryRethrowException(@Nullable Throwable e) throws Exception {

Review Comment:
   If the `throws Exception` is modified to `throws Throwable`, many classes that use the tryRethrowException have to `throws Throwable` 🤔 I'd prefer to refactor this in the old way and keep it in a separate hotfix commit. WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -77,7 +77,14 @@ public JobDispatcherLeaderProcessFactory createFactory(
         }
 
         final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore();
-        final Collection<JobResult> recoveredDirtyJobResults = getDirtyJobResults(jobResultStore);
+        final Collection<JobResult> recoveredDirtyJobResults;

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283937477


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########


Review Comment:
   Done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1284412779


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -561,21 +563,28 @@ public CompletableFuture<Acknowledge> submitFailedJob(
         return archiveExecutionGraphToHistoryServer(executionGraphInfo);
     }
 
+    /**
+     * Checks whether the given job has already been submitted, executed, or awaiting termination.
+     *
+     * @param jobId identifying the submitted job
+     * @return true if the job has already been submitted (is running) or has been executed
+     * @throws Exception if the job scheduling status cannot be retrieved
+     */
+    private boolean isDuplicateJob(JobID jobId) throws Exception {
+        return isInGloballyTerminalState(jobId).get()
+                || jobManagerRunnerRegistry.isRegistered(jobId)
+                || submittedAndWaitingTerminationJobIDs.contains(jobId);
+    }
+

Review Comment:
   This was accidentally re-added when rebasing to most-recent `master`. It can be removed again.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -112,10 +141,15 @@ private void withWriteLock(ThrowingRunnable<IOException> runnable) throws IOExce
         }
     }
 
-    private <T> T withReadLock(SupplierWithException<T, IOException> runnable) throws IOException {
+    private <T> CompletableFuture<T> withReadLockAsync(
+            SupplierWithException<T, IOException> runnable) {

Review Comment:
   ```suggestion
               SupplierWithException<T, IOException> supplier) {
   ```
   copy&paste error on my side. You changed it for `withReadLock` already. :+1: 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########


Review Comment:
   The commit message should have a `[hotfix][runtime]` prefix instead of being labeled with `[FLINK-27204]`. The change is indepenent of FLINK-27204 (i.e. more like a code cleanup that is shipped along with FLINK-27204). Think of it like that: The hotfix commit wouldn't need to be reverted when reverting FLINK-27204 because it's still a valid change to improve the code.
   
   Additionally, you only cleaned `markResultAsCleanInternal` into the hotfix commit. `hasDirtyJobResultEntryInternal`, `hasCleanJobResultEntryInternal` and `getDirtyResultsInternal` were cleaned as well but in the wrong commit. These three method changes should end up in the hotfix commit as well.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        dirtyResultAsync.get();
         assertThat(emptyBaseDirectory).exists().isDirectory();

Review Comment:
   I noticed that we could extend the tests to check the async nature: We could put the same (but inverted) assert in front of the trigger statement. That way we check that no synchronous activity happens. The same also applies to the other tests.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/JobResultStoreContractTest.java:
##########
@@ -46,72 +47,97 @@ public interface JobResultStoreContractTest {
     @Test
     default void testStoreJobResultsWithDuplicateIDsThrowsException() throws IOException {
         JobResultStore jobResultStore = createJobResultStore();
-        jobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        jobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY).join();
         final JobResultEntry otherEntryWithDuplicateId =
                 new JobResultEntry(
                         TestingJobResultStore.createSuccessfulJobResult(
                                 DUMMY_JOB_RESULT_ENTRY.getJobId()));
-        assertThatThrownBy(() -> jobResultStore.createDirtyResult(otherEntryWithDuplicateId))
-                .isInstanceOf(IllegalStateException.class);
+        assertThatThrownBy(
+                        () ->
+                                jobResultStore
+                                        .createDirtyResultAsync(otherEntryWithDuplicateId)
+                                        .join())
+                .isInstanceOf(CompletionException.class);

Review Comment:
   You didn't apply this in all appearances in `JobResultStoreContractTest`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,62 +44,62 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The future will be completed with {@link IllegalStateException} if the
+     *     passed {@code jobResultEntry} has a {@code JobID} attached that is already registered in
+     *     this {@code JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future can
+     *     also completed with {@link NoSuchElementException}. i.e. there is no corresponding {@code

Review Comment:
   ```suggestion
        * @return a successfully completed future if the result is marked successfully. The future can
        *     complete exceptionally with a {@link NoSuchElementException}. i.e. there is no corresponding {@code
   ```
   nitty nit: we might want to make it more explicit that the future completes "exceptionally"



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,66 +46,92 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
 
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
+        this.ioExecutor = ioExecutor;
+    }
+
+    @Override
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .thenAccept(
+                        hasJobResultEntry ->
+                                Preconditions.checkState(
+                                        !hasJobResultEntry,
+                                        "Job result store already contains an entry for job %s",
+                                        jobResultEntry.getJobId()))
+                .thenCompose(
+                        ignoredVoid ->
+                                withWriteLockAsync(
+                                        () -> createDirtyResultInternal(jobResultEntry)));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);
-            return;
-        }
-
-        withWriteLock(() -> markResultAsCleanInternal(jobId));
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        return hasCleanJobResultEntryAsync(jobId)
+                .thenCompose(
+                        hasCleanJobResultEntry -> {
+                            if (hasCleanJobResultEntry) {
+                                LOG.debug(
+                                        "The job {} is already marked as clean. No action required.",
+                                        jobId);
+                                return FutureUtils.completedVoidFuture();
+                            }
+
+                            return withWriteLockAsync(() -> markResultAsCleanInternal(jobId));
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void markResultAsCleanInternal(JobID jobId)
             throws IOException, NoSuchElementException;
 
     @Override
-    public boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(
+    public CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(
                 () ->
                         hasDirtyJobResultEntryInternal(jobId)
                                 || hasCleanJobResultEntryInternal(jobId));
     }
 
     @Override
-    public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasDirtyJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasDirtyJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
-    public boolean hasCleanJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasCleanJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasCleanJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
     public Set<JobResult> getDirtyResults() throws IOException {
-        return withReadLock(this::getDirtyResultsInternal);
+        return getDirtyResultsInternal();

Review Comment:
   ```suggestion
           return withReadLock(this::getDirtyResultsInternal());
   ```
   Yikes, that's my bad: We should remove the lock here.



##########
flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java:
##########
@@ -381,9 +381,12 @@ public static void rethrowException(Throwable t) throws Exception {
      * @param e exception to throw if not null.
      * @throws Exception
      */
-    public static void tryRethrowException(@Nullable Exception e) throws Exception {
+    public static void tryRethrowException(@Nullable Throwable e) throws Exception {

Review Comment:
   We can revert the change of `ExceptionUtils`. We're not using it anymore, as far as I can tell.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        dirtyResultAsync.get();

Review Comment:
   ```suggestion
       FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
   ```
   nit: `FlinkAssertions.assertThatFuture` is a handy utility to make future-based testing more readable.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -54,11 +57,15 @@ public class FileSystemJobResultStoreFileOperationsTest {
 
     private Path basePath;
 
+    final ManuallyTriggeredScheduledExecutor manuallyTriggeredExecutor =

Review Comment:
   ```suggestion
       private final ManuallyTriggeredScheduledExecutor manuallyTriggeredExecutor =
   ```
   Looks like that one slipped through in the previous review. Is there a reason why we don't make that one `private`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,66 +46,92 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
 
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
+        this.ioExecutor = ioExecutor;
+    }
+
+    @Override
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .thenAccept(
+                        hasJobResultEntry ->
+                                Preconditions.checkState(
+                                        !hasJobResultEntry,
+                                        "Job result store already contains an entry for job %s",
+                                        jobResultEntry.getJobId()))
+                .thenCompose(
+                        ignoredVoid ->
+                                withWriteLockAsync(
+                                        () -> createDirtyResultInternal(jobResultEntry)));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);
-            return;
-        }
-
-        withWriteLock(() -> markResultAsCleanInternal(jobId));
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        return hasCleanJobResultEntryAsync(jobId)
+                .thenCompose(
+                        hasCleanJobResultEntry -> {
+                            if (hasCleanJobResultEntry) {
+                                LOG.debug(
+                                        "The job {} is already marked as clean. No action required.",
+                                        jobId);
+                                return FutureUtils.completedVoidFuture();
+                            }
+
+                            return withWriteLockAsync(() -> markResultAsCleanInternal(jobId));
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void markResultAsCleanInternal(JobID jobId)
             throws IOException, NoSuchElementException;
 
     @Override
-    public boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(
+    public CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(
                 () ->
                         hasDirtyJobResultEntryInternal(jobId)
                                 || hasCleanJobResultEntryInternal(jobId));
     }
 
     @Override
-    public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasDirtyJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasDirtyJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
-    public boolean hasCleanJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasCleanJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasCleanJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
     public Set<JobResult> getDirtyResults() throws IOException {
-        return withReadLock(this::getDirtyResultsInternal);
+        return getDirtyResultsInternal();
     }
 
     @GuardedBy("readWriteLock")
     protected abstract Set<JobResult> getDirtyResultsInternal() throws IOException;
 
+    private CompletableFuture<Void> withWriteLockAsync(ThrowingRunnable<IOException> runnable) {
+        return FutureUtils.supplyAsync(

Review Comment:
   ```suggestion
           return FutureUtils.runAsync(
   ```
   `runAsync` is good enough here. We're not returning any value.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295642610


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

Review Comment:
   Moving the `runIfValidLeader` method up destroys the git history to some degree (imagine someone wants to see the history of this specific method in Intellij by selecting the method and using the "Git > Show history for Selection..." feature). Instead, why not refactor the method in its original location and add the `supplyAsyncIfValidLeader` next to that one.



##########
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:
   a bit out-of-scope, but: I feel like these log statement should be in debug level. :thinking: WDYT? But we should fix it in a separate hotfix commit if you agree.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {

Review Comment:
   ~The if (hasJobResult == null) { block becomes obsolete here (because there shouldn't be a situation where no result is passed if no error happened in upstream async calls for handle(BiFunction). You could add a Precondition.checkNotNull if you like. But the subsequent if call would cause a NullPointerException anyway which makes the precondition obsolete again.~
   
   ~(... and a side-remark here: we would at least add logs rather than only returning null in case if the if (hasJobResult == null) { case.)~
   
   Ok, I misread the code here. The `null` check actually has a meaning here because the upstream call of `supplyAsyncIfValidLeader` call returns a `CompletableFuture` that completed successfully with `null` if the leadership is not granted anymore (the `noLeaderFallback`). This functional dependency is a bit hidden (IMHO). I see two options to improve the code:
   1. Add a comment to the `if (hasJobResult == null) {` explaining why this is needed. I would also like to add a `@Nullable` annotation to `printLogIfNotValidLeader` to acknowlegde. But I don't know how to handle such a case with async calls. The `@Nullable` annotation becomes kind of ambigious.
   2. Consider not having the leadership in the `supplyAsyncIfValidLeader` an error scenario that shall be handled in the downstream `handle` call. This we can achieve by returning an exceptionally completed future. You might want to introduce a new exception `LeadershipLostException` that derives from `LeaderElectionException` to cover that error scenario. You could add the log message in the `handle` method when handling `throwable != null`, something like:
   ```
   if (throwable instanceof LeadershipLostException) {
       // print your log statement
   } else if (throwable != null) {
       ExceptionUtils.rethrow(throwable);
   }
   ```
   
   I'm in favor of the 2nd solution because it makes the code more explicit and, in this regard, easier to understand.
   



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1302560535


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   You didn't do a pull before adding the changes (I did a [force-push](https://github.com/apache/flink/pull/22341#pullrequestreview-1588695382) to include a few minor changes previous). These changes were reverted with your most-recent push



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1301127750


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -515,7 +516,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
 
         try {
-            if (isInGloballyTerminalState(jobID)) {
+            if (isInGloballyTerminalState(jobID).get()) {

Review Comment:
   The change looks good. :+1: Please squash the commits and I will do a final pass over it.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1301628816


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   ```suggestion
                               } else if (jobManagerRunnerRegistry.isRegistered(jobID)
   ```
   Ok, that's a tricky one which I missed: The Dispatcher has one requirement: Any state access needs to happen in the main thread of the Dispatcher. There's a special implementation of `JobManagerRunnerRegistry` that ensures this invariant (see [OnMainThreadJobmanagerRunnerRegistry](https://github.com/apache/flink/blob/3efd4c2cf1be670f499e6637445e283e48deee60/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/OnMainThreadJobManagerRunnerRegistry.java)). Our change (with the `thenCompose` being chained with the future that's returned by `isInGloballyTerminalState`) is going against this requirement. Why?
   
   `isInGloballyTerminalState` calls `jobResultStore.hasJobResultEntryAsync` which executes the logic on the `ioExecutor` (i.e. a thread for IO operations which is not the Dispatcher's main thread) internally. The returned future is linked to this executor, i.e. any chained `CompletableFuture` calls will run in the same thread. The `thenCompose` logic is, therefore, also executed in the `ioExecutor` instead of the main thread.
   
   To workaround this, we have to change the executor for the chained execution. This can be achieved by using `thenComposeAsync`, instead. Here we would specify the main thread executor by calling `getMainThreadExecutor`. One example where it's done like that is [Dispatcher:619](https://github.com/apache/flink/blob/b6992b9d80e8ca9c6a81d198d9ed628821e4ab49/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java#L619): The `cleanupAsync` method is executed on the `ioExecutor`. But the error handling has to happen in the main thread again. That's where we use `handleAsync` with `getMainThreadExecutor`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264363422


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,87 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {

Review Comment:
   The  parameter of `ExceptionUtils.tryRethrowException` is `Exception` and there is a `Throwable`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264363969


##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));
                                                     }
+                                                    result.complete(true);
+                                                    return result;

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264359023


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -103,8 +103,8 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
 
     private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   I think adding the method `getDirtyJobResult ` is reasonable.  Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -191,8 +191,8 @@ private Collection<JobResult> getDirtyJobResultsIfRunning() {
 
     private Collection<JobResult> getDirtyJobResults() {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1269644251


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -103,8 +103,8 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
 
     private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Is there a reason why you kept the asynchronous version of the method?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283946578


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.trigger();

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1284120242


##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java:
##########
@@ -116,58 +111,53 @@ public static TestingJobResultStore.Builder builder() {
     /** {@code Builder} for instantiating {@code TestingJobResultStore} instances. */
     public static class Builder {
 
-        private ThrowingConsumer<JobResultEntry, ? extends IOException> createDirtyResultConsumer =
-                ignored -> {};
-        private ThrowingConsumer<JobID, ? extends IOException> markResultAsCleanConsumer =
-                ignored -> {};
+        private Function<JobResultEntry, CompletableFuture<Void>> createDirtyResultConsumer =
+                jobResultEntry -> CompletableFuture.completedFuture(null);
+        private Function<JobID, CompletableFuture<Void>> markResultAsCleanConsumer =
+                jobID -> CompletableFuture.completedFuture(null);
 
-        private FunctionWithException<JobID, Boolean, ? extends IOException>
-                hasJobResultEntryFunction = ignored -> false;
-        private FunctionWithException<JobID, Boolean, ? extends IOException>
-                hasDirtyJobResultEntryFunction = ignored -> false;
-        private FunctionWithException<JobID, Boolean, ? extends IOException>
-                hasCleanJobResultEntryFunction = ignored -> false;
+        private Function<JobID, CompletableFuture<Boolean>> hasJobResultEntryFunction =
+                jobID -> CompletableFuture.completedFuture(false);
+        private Function<JobID, CompletableFuture<Boolean>> hasDirtyJobResultEntryFunction =
+                jobID -> CompletableFuture.completedFuture(false);
+        private Function<JobID, CompletableFuture<Boolean>> hasCleanJobResultEntryFunction =
+                jobID -> CompletableFuture.completedFuture(false);
 
-        private SupplierWithException<Set<JobResult>, ? extends IOException>
-                getDirtyResultsSupplier = Collections::emptySet;
+        private Supplier<Set<JobResult>> dirtyResultsSupplier = Collections::emptySet;
 
         public Builder withCreateDirtyResultConsumer(
-                ThrowingConsumer<JobResultEntry, ? extends IOException> createDirtyResultConsumer) {
+                Function<JobResultEntry, CompletableFuture<Void>> createDirtyResultConsumer) {
             this.createDirtyResultConsumer = createDirtyResultConsumer;
             return this;
         }
 
         public Builder withMarkResultAsCleanConsumer(
-                ThrowingConsumer<JobID, ? extends IOException> markResultAsCleanConsumer) {
+                Function<JobID, CompletableFuture<Void>> markResultAsCleanConsumer) {
             this.markResultAsCleanConsumer = markResultAsCleanConsumer;
             return this;
         }
 
         public Builder withHasJobResultEntryFunction(
-                FunctionWithException<JobID, Boolean, ? extends IOException>
-                        hasJobResultEntryFunction) {
+                Function<JobID, CompletableFuture<Boolean>> hasJobResultEntryFunction) {
             this.hasJobResultEntryFunction = hasJobResultEntryFunction;
             return this;
         }
 
         public Builder withHasDirtyJobResultEntryFunction(
-                FunctionWithException<JobID, Boolean, ? extends IOException>
-                        hasDirtyJobResultEntryFunction) {
+                Function<JobID, CompletableFuture<Boolean>> hasDirtyJobResultEntryFunction) {
             this.hasDirtyJobResultEntryFunction = hasDirtyJobResultEntryFunction;
             return this;
         }
 
         public Builder withHasCleanJobResultEntryFunction(
-                FunctionWithException<JobID, Boolean, ? extends IOException>
-                        hasCleanJobResultEntryFunction) {
+                Function<JobID, CompletableFuture<Boolean>> hasCleanJobResultEntryFunction) {
             this.hasCleanJobResultEntryFunction = hasCleanJobResultEntryFunction;
             return this;
         }
 
         public Builder withGetDirtyResultsSupplier(
-                SupplierWithException<Set<JobResult>, ? extends IOException>
-                        getDirtyResultsSupplier) {
-            this.getDirtyResultsSupplier = getDirtyResultsSupplier;
+                Supplier<Set<JobResult>> getDirtyResultsSupplier) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283019274


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,61 +47,126 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
+
+    public AbstractThreadsafeJobResultStore(Executor ioExecutor) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1296742543


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

Review Comment:
   1. The `runIfValidLeader ` has been modified in its original location.
   2. The `supplyAsyncIfValidLeader ` has been added next to `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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288764461


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,80 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        assertThat(emptyBaseDirectory).doesNotExist();
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
         assertThat(emptyBaseDirectory).exists().isDirectory();
     }
 
     @Test
     public void testStoreDirtyJobResultCreatesFile() throws Exception {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).doesNotExist();
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
         assertThat(getCleanResultIdsFromFileSystem()).isEmpty();
         assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).exists().isFile().isNotEmpty();
     }
 
     @Test
     public void testStoreCleanJobResultCreatesFile() throws Exception {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
+        CompletableFuture<Void> markCleanAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        assertThat(getCleanResultIdsFromFileSystem())
+                .doesNotContain(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(markCleanAsync).eventuallySucceeds();
         assertThat(getCleanResultIdsFromFileSystem())
                 .containsExactlyInAnyOrder(DUMMY_JOB_RESULT_ENTRY.getJobId());
     }
 
     @Test
-    public void testStoreCleanJobResultDeletesDirtyFile() throws Exception {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+    public void testStoreCleanJobResultDeletesDirtyFile() {
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).doesNotExist();
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
         assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).exists().isFile().isNotEmpty();
 
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        CompletableFuture<Void> markResultAsCleanAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(markResultAsCleanAsync).eventuallySucceeds();
         assertThat(expectedDirtyFile(DUMMY_JOB_RESULT_ENTRY)).doesNotExist();
     }
 
     @Test
     public void testCleanDirtyJobResultTwiceIsIdempotent() throws IOException {
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
-
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds();
+        CompletableFuture<Void> cleanResultAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        manuallyTriggeredExecutor.triggerAll();
+        FlinkAssertions.assertThatFuture(cleanResultAsync).eventuallySucceeds();
         final byte[] cleanFileData =
                 FileUtils.readAllBytes(expectedCleanFile(DUMMY_JOB_RESULT_ENTRY).toPath());
 
-        fileSystemJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        CompletableFuture<Void> markResultAsCleanAsync =
+                fileSystemJobResultStore.markResultAsCleanAsync(DUMMY_JOB_RESULT_ENTRY.getJobId());
+        assertThat(expectedCleanFile(DUMMY_JOB_RESULT_ENTRY))
+                .doesNotHaveSameHashCodeAs(cleanFileData);

Review Comment:
   I don't get this assert? Why do we check this here? Essentially, what we want to check is that the modification date of the file is the same after each `markResultAsCleanAsync` call. :thinking: 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295631119


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {

Review Comment:
   The `if (hasJobResult == null) {` block becomes obsolete here (because there shouldn't be a situation where no result is passed if no error happened in upstream async calls for `handle(BiFunction)`. You could add a `Precondition.checkNotNull` if you like. But the subsequent if call would cause a `NullPointerException` anyway which makes the precondition obsolete again.
   
   (... and a side-remark here: we would at least add logs rather than only returning `null` in case if the `if (hasJobResult == null) {` case.)



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1293134048


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {

Review Comment:
   This would swallow a potential IOException of the `hasJobResultEntryAsync` call. The old implementation handled these kind of errors. PTAL



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1302585103


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   😼 I have added the changes in the [comparison](https://github.com/apache/flink/compare/5a7329d02ce51f53bfd4ab8c2f6e30b5271c539f..ee05e69244ad3254c05a567e37e6be7d85ce0881). Really sorry for this.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   😼 I have added the changes in the [comparison](https://github.com/apache/flink/compare/5a7329d02ce51f53bfd4ab8c2f6e30b5271c539f..ee05e69244ad3254c05a567e37e6be7d85ce0881). Really sorry for this.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1302622261


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -513,36 +514,39 @@ private void stopDispatcherServices() throws Exception {
     public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout) {
         final JobID jobID = jobGraph.getJobID();
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
-
-        try {
-            if (isInGloballyTerminalState(jobID)) {
-                log.warn(
-                        "Ignoring JobGraph submission '{}' ({}) because the job already reached a globally-terminal state (i.e. {}) in a previous execution.",
-                        jobGraph.getName(),
-                        jobID,
-                        Arrays.stream(JobStatus.values())
-                                .filter(JobStatus::isGloballyTerminalState)
-                                .map(JobStatus::name)
-                                .collect(Collectors.joining(", ")));
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.ofGloballyTerminated(jobID));
-            } else if (jobManagerRunnerRegistry.isRegistered(jobID)
-                    || submittedAndWaitingTerminationJobIDs.contains(jobID)) {
-                // job with the given jobID is not terminated, yet
-                return FutureUtils.completedExceptionally(
-                        DuplicateJobSubmissionException.of(jobID));
-            } else if (isPartialResourceConfigured(jobGraph)) {
-                return FutureUtils.completedExceptionally(
-                        new JobSubmissionException(
-                                jobID,
-                                "Currently jobs is not supported if parts of the vertices have "
-                                        + "resources configured. The limitation will be removed in future versions."));
-            } else {
-                return internalSubmitJob(jobGraph);
-            }
-        } catch (FlinkException e) {
-            return FutureUtils.completedExceptionally(e);
-        }
+        return isInGloballyTerminalState(jobID)
+                .thenCompose(
+                        isTerminated -> {
+                            if (isTerminated) {
+                                log.warn(
+                                        "Ignoring JobGraph submission '{}' ({}) because the job already "
+                                                + "reached a globally-terminal state (i.e. {}) in a "
+                                                + "previous execution.",
+                                        jobGraph.getName(),
+                                        jobID,
+                                        Arrays.stream(JobStatus.values())
+                                                .filter(JobStatus::isGloballyTerminalState)
+                                                .map(JobStatus::name)
+                                                .collect(Collectors.joining(", ")));
+                                return FutureUtils.completedExceptionally(
+                                        DuplicateJobSubmissionException.ofGloballyTerminated(
+                                                jobID));
+                            } else if (jobManagerRunnerRegistry.isRegistered(jobID)

Review Comment:
   No worries. Let's see whether CI becomes :green_circle: this time. :crossed_fingers: 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1554191565

   @XComp I have made a round of changes. Please take a look at it when you have time. 😃


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1543301331

   @XComp Thanks for your reply! 😀
   I only have a small questions. What should I define the return types of
   
   `void createDirtyResult(JobResultEntry) `
   `void markResultAsClean(JobID)`
   
   Currently the invokers of these two methods need sync behaviors. If we give it a Completable<Void>,and the invokers must use future.get(). From this point, refracting return types of these two methods is a little meaningless.
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1297931816


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

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288206790


##########
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreFileOperationsTest.java:
##########
@@ -116,46 +123,74 @@ public void testBaseDirectoryCreationOnResultStoreInitialization() throws Except
         assertThat(emptyBaseDirectory).doesNotExist();
 
         fileSystemJobResultStore =
-                new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false);
+                new FileSystemJobResultStore(
+                        basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor);
         // Result store operations are creating the base directory on-the-fly
         assertThat(emptyBaseDirectory).doesNotExist();
-        fileSystemJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY);
+        CompletableFuture<Void> dirtyResultAsync =
+                fileSystemJobResultStore.createDirtyResultAsync(DUMMY_JOB_RESULT_ENTRY);
+        manuallyTriggeredExecutor.triggerAll();
+        dirtyResultAsync.get();
         assertThat(emptyBaseDirectory).exists().isDirectory();

Review Comment:
   I have add inverted asserts in the tests in FileSystemJobResultStoreFileOperationsTest.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1301133436


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -338,32 +356,24 @@ private void confirmLeadership(
         FutureUtils.assertNoException(
                 leaderAddressFuture.thenAccept(
                         address -> {
-                            synchronized (lock) {
-                                if (isValidLeader(leaderSessionId)) {
-                                    LOG.debug("Confirm leadership {}.", leaderSessionId);
-                                    leaderElection.confirmLeadership(leaderSessionId, address);
-                                } else {
-                                    LOG.trace(
-                                            "Ignore confirming leadership because the leader {} is no longer valid.",
-                                            leaderSessionId);
-                                }
-                            }
+                            runIfValidLeader(
+                                    leaderSessionId,
+                                    () -> {
+                                        LOG.debug("Confirm leadership {}.", leaderSessionId);
+                                        leaderElection.confirmLeadership(leaderSessionId, address);
+                                    },
+                                    "confirming leadership");
                         }));
     }
 
     private void forwardResultFuture(
             UUID leaderSessionId, CompletableFuture<JobManagerRunnerResult> resultFuture) {
         resultFuture.whenComplete(
                 (jobManagerRunnerResult, throwable) -> {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp merged pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp merged PR #22341:
URL: https://github.com/apache/flink/pull/22341


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295337613


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -488,18 +523,21 @@ private boolean isRunning() {
         return state == State.RUNNING;
     }
 
-    private void runIfValidLeader(
-            UUID expectedLeaderId, Runnable action, String actionDescription) {
+    private <T> CompletableFuture<T> runIfValidLeader(

Review Comment:
   I've added these two methods.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(

Review Comment:
   Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")

Review Comment:
   Sorry for the mistake. Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {

Review Comment:
   Currently the `throwable` is handled.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288141874


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,66 +46,92 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
 
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
+        this.ioExecutor = ioExecutor;
+    }
+
+    @Override
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .thenAccept(
+                        hasJobResultEntry ->
+                                Preconditions.checkState(
+                                        !hasJobResultEntry,
+                                        "Job result store already contains an entry for job %s",
+                                        jobResultEntry.getJobId()))
+                .thenCompose(
+                        ignoredVoid ->
+                                withWriteLockAsync(
+                                        () -> createDirtyResultInternal(jobResultEntry)));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);
-            return;
-        }
-
-        withWriteLock(() -> markResultAsCleanInternal(jobId));
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        return hasCleanJobResultEntryAsync(jobId)
+                .thenCompose(
+                        hasCleanJobResultEntry -> {
+                            if (hasCleanJobResultEntry) {
+                                LOG.debug(
+                                        "The job {} is already marked as clean. No action required.",
+                                        jobId);
+                                return FutureUtils.completedVoidFuture();
+                            }
+
+                            return withWriteLockAsync(() -> markResultAsCleanInternal(jobId));
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void markResultAsCleanInternal(JobID jobId)
             throws IOException, NoSuchElementException;
 
     @Override
-    public boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(
+    public CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(
                 () ->
                         hasDirtyJobResultEntryInternal(jobId)
                                 || hasCleanJobResultEntryInternal(jobId));
     }
 
     @Override
-    public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasDirtyJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasDirtyJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
-    public boolean hasCleanJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasCleanJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasCleanJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
     public Set<JobResult> getDirtyResults() throws IOException {
-        return withReadLock(this::getDirtyResultsInternal);
+        return getDirtyResultsInternal();

Review Comment:
   The lock is added now.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283100724


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1390,29 +1392,71 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
+        CompletableFuture<Boolean> shouldCheckDirtyJobResult =
+                jobResultStore
+                        .hasCleanJobResultEntryAsync(jobId)
+                        .handleAsync(
+                                (hasCleanJobResultEntry, throwable) -> {
+                                    if (throwable != null) {
+                                        writeFuture.completeExceptionally(throwable);
+                                        return false;
+                                    } else {
+                                        if (hasCleanJobResultEntry) {
+                                            log.warn(
+                                                    "Job {} is already marked as clean but "
+                                                            + "clean up was triggered again.",
+                                                    jobId);
+                                            writeFuture.complete(null);
+                                            return false;
+                                        } else {
+                                            return true;
+                                        }
+                                    }
+                                });
+        shouldCheckDirtyJobResult.whenCompleteAsync(
+                (shouldCheck, throwable1) -> {
+                    if (throwable1 != null) {
+                        writeFuture.completeExceptionally(throwable1);
                         return;
                     }
-                    writeFuture.complete(null);
+                    if (shouldCheck) {
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(jobId)
+                                .whenCompleteAsync(
+                                        (hasDirtyJobResultEntry, throwable2) -> {
+                                            if (throwable2 != null) {
+                                                writeFuture.completeExceptionally(throwable2);
+                                                return;
+                                            }
+                                            if (!hasDirtyJobResultEntry) {
+                                                jobResultStore
+                                                        .createDirtyResultAsync(
+                                                                new JobResultEntry(
+                                                                        JobResult.createFrom(
+                                                                                archivedExecutionGraph)))
+                                                        .whenCompleteAsync(
+                                                                (unused, throwable3) -> {
+                                                                    if (throwable3 != null) {
+                                                                        writeFuture
+                                                                                .completeExceptionally(
+                                                                                        throwable3);
+                                                                        return;
+                                                                    }
+                                                                    log.info(
+                                                                            "Job {} has been registered "
+                                                                                    + "for cleanup in "
+                                                                                    + "the JobResultStore "
+                                                                                    + "after reaching a "
+                                                                                    + "terminal state.",
+                                                                            jobId);
+                                                                    writeFuture.complete(null);
+                                                                });
+                                            } else {
+                                                writeFuture.complete(null);
+                                            }
+                                        });
+                    }

Review Comment:
   Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,61 +47,126 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
+
+    public AbstractThreadsafeJobResultStore(Executor ioExecutor) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283927882


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean}
+     *     {@link JobResultEntry} exists for the given {@code JobID}; otherwise a successfully
+     *     completed future with {@code false}.

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283934746


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,70 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The method will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future will
+     *     completed with {@link NoSuchElementException} if there is no corresponding {@code dirty}
+     *     job present in the store for the given {@code JobID}.

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283935250


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########
@@ -57,17 +61,17 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283944785


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -277,18 +277,11 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
 
     @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);
+            throws FlinkException, ExecutionException, InterruptedException {
+        if (jobResultStore.hasJobResultEntryAsync(getJobID()).get()) {

Review Comment:
   Yes, I agree that make both `startJobMasterServiceProcessAsync` and `verifyJobSchedulingStatusAndCreateJobMasterServiceProcess` leverage the async nature is important. 
   Currently dealing with the sync and async future logic in these methods will be complex and make the code hard to understand, so refactoring this needs to be well designed and may be completed in another pr, WDYT?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1297920993


##########
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:
   These codes have been 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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1282185399


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########


Review Comment:
   Maybe it helps to compare your approach with the [proposal above](https://github.com/apache/flink/pull/22341#discussion_r1282183101) to get a better understanding of the `CompletableFuture` usage. Feel free to ask questions about it. And don't hesitate to question my approach! See it as a discussion item. :-)



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1576632397

   @XComp I have made a round of changes. Please take a look at it when you have time. 😄


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1270162450


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -191,8 +191,8 @@ private Collection<JobResult> getDirtyJobResultsIfRunning() {
 
     private Collection<JobResult> getDirtyJobResults() {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1270192837


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -575,12 +576,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      */
     private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
         try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
+            return jobResultStore.hasJobResultEntryAsync(jobId).get();

Review Comment:
   Sorry for the incorrect usage of `CompletableFuture `. I've updated the code.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1663693669

   Another feedback on reviews: You don't need to squash all the commits together while the review is ongoing. Usually it helps the reviewer to see the individual changes per commit (e.g. a new commit after a review shows the changes based on the review). Squashing everything together makes us lose this information. As a consequence, the reviewer might have to go over already reviewed code changes again.
   
   You can collect all the changes in one diff still without having them squashed into a single commit through git. Therefore, squashing the commits into a single one doesn't bring any value as far as I can see. ...just as a hint for future changes. :-)


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283935357


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java:
##########
@@ -57,17 +61,17 @@ public void markResultAsCleanInternal(JobID jobId) throws IOException, NoSuchEle
     }
 
     @Override
-    public boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasDirtyJobResultEntryInternal(JobID jobId) {
         return dirtyJobResults.containsKey(jobId);
     }
 
     @Override
-    public boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException {
+    public Boolean hasCleanJobResultEntryInternal(JobID jobId) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283010571


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1273,7 +1270,22 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
         if (cleanupJobState.isGlobalCleanup()) {
             return globalResourceCleaner
                     .cleanupAsync(jobId)
-                    .thenRunAsync(() -> markJobAsClean(jobId), ioExecutor)
+                    .thenCompose(unused -> jobResultStore.markResultAsCleanAsync(jobId))
+                    .handle(
+                            (BiFunction<Void, Throwable, Void>)
+                                    (unused, e) -> {

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283018521


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1390,29 +1392,71 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
+        CompletableFuture<Boolean> shouldCheckDirtyJobResult =
+                jobResultStore
+                        .hasCleanJobResultEntryAsync(jobId)
+                        .handleAsync(
+                                (hasCleanJobResultEntry, throwable) -> {
+                                    if (throwable != null) {
+                                        writeFuture.completeExceptionally(throwable);
+                                        return false;
+                                    } else {
+                                        if (hasCleanJobResultEntry) {
+                                            log.warn(
+                                                    "Job {} is already marked as clean but "
+                                                            + "clean up was triggered again.",
+                                                    jobId);
+                                            writeFuture.complete(null);
+                                            return false;
+                                        } else {
+                                            return true;
+                                        }
+                                    }
+                                });
+        shouldCheckDirtyJobResult.whenCompleteAsync(
+                (shouldCheck, throwable1) -> {
+                    if (throwable1 != null) {
+                        writeFuture.completeExceptionally(throwable1);
                         return;
                     }
-                    writeFuture.complete(null);
+                    if (shouldCheck) {
+                        jobResultStore
+                                .hasDirtyJobResultEntryAsync(jobId)
+                                .whenCompleteAsync(
+                                        (hasDirtyJobResultEntry, throwable2) -> {
+                                            if (throwable2 != null) {
+                                                writeFuture.completeExceptionally(throwable2);
+                                                return;
+                                            }
+                                            if (!hasDirtyJobResultEntry) {
+                                                jobResultStore
+                                                        .createDirtyResultAsync(
+                                                                new JobResultEntry(
+                                                                        JobResult.createFrom(
+                                                                                archivedExecutionGraph)))
+                                                        .whenCompleteAsync(
+                                                                (unused, throwable3) -> {
+                                                                    if (throwable3 != null) {
+                                                                        writeFuture
+                                                                                .completeExceptionally(
+                                                                                        throwable3);
+                                                                        return;
+                                                                    }
+                                                                    log.info(
+                                                                            "Job {} has been registered "
+                                                                                    + "for cleanup in "
+                                                                                    + "the JobResultStore "
+                                                                                    + "after reaching a "
+                                                                                    + "terminal state.",
+                                                                            jobId);
+                                                                    writeFuture.complete(null);
+                                                                });
+                                            } else {
+                                                writeFuture.complete(null);
+                                            }
+                                        });
+                    }

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1284121890


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########


Review Comment:
   I've followed the code of your version.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1283945744


##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -579,10 +587,15 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws
         final CompletableFuture<JobResultEntry> dirtyJobFuture = new CompletableFuture<>();
         final JobResultStore jobResultStore =
                 TestingJobResultStore.builder()
-                        .withCreateDirtyResultConsumer(dirtyJobFuture::complete)
+                        .withCreateDirtyResultConsumer(
+                                jobResultEntry -> {
+                                    dirtyJobFuture.complete(jobResultEntry);
+                                    return FutureUtils.completedVoidFuture();
+                                })
                         .withMarkResultAsCleanConsumer(
                                 jobId -> {
-                                    throw new IOException("Expected IOException.");
+                                    return FutureUtils.completedExceptionally(

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1288179888


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,62 +44,62 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return a successfully completed future with {@code true} if the dirty result is created
+     *     successfully. The future will be completed with {@link IllegalStateException} if the
+     *     passed {@code jobResultEntry} has a {@code JobID} attached that is already registered in
+     *     this {@code JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
-     *     store for the given {@code JobID}.
+     * @return a successfully completed future if the result is marked successfully, The future can
+     *     also completed with {@link NoSuchElementException}. i.e. there is no corresponding {@code

Review Comment:
   I've fixed the note.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1284435068


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -43,66 +46,92 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
 
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
+    private final Executor ioExecutor;
 
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    protected AbstractThreadsafeJobResultStore(Executor ioExecutor) {
+        this.ioExecutor = ioExecutor;
+    }
+
+    @Override
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .thenAccept(
+                        hasJobResultEntry ->
+                                Preconditions.checkState(
+                                        !hasJobResultEntry,
+                                        "Job result store already contains an entry for job %s",
+                                        jobResultEntry.getJobId()))
+                .thenCompose(
+                        ignoredVoid ->
+                                withWriteLockAsync(
+                                        () -> createDirtyResultInternal(jobResultEntry)));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);
-            return;
-        }
-
-        withWriteLock(() -> markResultAsCleanInternal(jobId));
+    public CompletableFuture<Void> markResultAsCleanAsync(JobID jobId) {
+        return hasCleanJobResultEntryAsync(jobId)
+                .thenCompose(
+                        hasCleanJobResultEntry -> {
+                            if (hasCleanJobResultEntry) {
+                                LOG.debug(
+                                        "The job {} is already marked as clean. No action required.",
+                                        jobId);
+                                return FutureUtils.completedVoidFuture();
+                            }
+
+                            return withWriteLockAsync(() -> markResultAsCleanInternal(jobId));
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void markResultAsCleanInternal(JobID jobId)
             throws IOException, NoSuchElementException;
 
     @Override
-    public boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(
+    public CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(
                 () ->
                         hasDirtyJobResultEntryInternal(jobId)
                                 || hasCleanJobResultEntryInternal(jobId));
     }
 
     @Override
-    public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasDirtyJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasDirtyJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
-    public boolean hasCleanJobResultEntry(JobID jobId) throws IOException {
-        return withReadLock(() -> hasCleanJobResultEntryInternal(jobId));
+    public CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId) {
+        return withReadLockAsync(() -> hasCleanJobResultEntryInternal(jobId));
     }
 
     @GuardedBy("readWriteLock")
     protected abstract boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException;
 
     @Override
     public Set<JobResult> getDirtyResults() throws IOException {
-        return withReadLock(this::getDirtyResultsInternal);
+        return getDirtyResultsInternal();

Review Comment:
   ```suggestion
           return withReadLock(this::getDirtyResultsInternal());
   ```
   Yikes, that's my bad: We shouldn't remove the lock here.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1295337267


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########
@@ -261,34 +261,69 @@ public void grantLeadership(UUID leaderSessionID) {
 
     @GuardedBy("lock")
     private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
-        sequentialOperation =
-                sequentialOperation.thenRun(
-                        () ->
-                                runIfValidLeader(
-                                        leaderSessionId,
-                                        ThrowingRunnable.unchecked(
-                                                () ->
-                                                        verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(
-                                                                leaderSessionId)),
-                                        "verify job scheduling status and create JobMasterServiceProcess"));
-
+        boolean isValid;
+        synchronized (lock) {
+            isValid = isValidLeader(leaderSessionId);
+        }
+        if (isValid) {
+            sequentialOperation =
+                    sequentialOperation.thenCompose(
+                            unused ->
+                                    runIfValidLeader(
+                                                    leaderSessionId,
+                                                    () ->
+                                                            jobResultStore.hasJobResultEntryAsync(
+                                                                    getJobID()),
+                                                    "verify jbb result entry")
+                                            .handle(
+                                                    (hasJobResult, throwable) -> {
+                                                        if (hasJobResult == null) {
+                                                            return null;
+                                                        }
+                                                        if (hasJobResult) {
+                                                            runIfValidLeader(

Review Comment:
   I've added these two methods: `handleJobAlreadyDoneIfValidLeader(leaderSessionID)` and `createNewJobMasterServiceProcessIfValidLeader(leaderSessionID))`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1300892311


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -515,7 +516,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
 
         try {
-            if (isInGloballyTerminalState(jobID)) {
+            if (isInGloballyTerminalState(jobID).get()) {

Review Comment:
   I've added a fixup commit. 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -515,7 +516,7 @@ public CompletableFuture<Acknowledge> submitJob(JobGraph jobGraph, Time timeout)
         log.info("Received JobGraph submission '{}' ({}).", jobGraph.getName(), jobID);
 
         try {
-            if (isInGloballyTerminalState(jobID)) {
+            if (isInGloballyTerminalState(jobID).get()) {

Review Comment:
   I've added a fixup commit. PTAL.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204][flink-runtime] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1300891908


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########


Review Comment:
   Sorry for the mistake.😐 Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java:
##########


Review Comment:
   Sorry for the mistake.😐 Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264360094


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -101,14 +109,9 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
         return new JobDispatcherLeaderProcessFactoryFactory(jobGraphRetriever);
     }
 
-    private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
-        try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
-            throw new FlinkRuntimeException(
-                    "Could not retrieve the JobResults of dirty jobs from the underlying JobResultStore.",
-                    e);
-        }
+    private static CompletableFuture<Set<JobResult>> getDirtyJobResultsAsync(

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264363543


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +43,67 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
-     * @throws IOException if the creation of the dirty result failed for IO reasons.
-     * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
-     *     attached that is already registered in this {@code JobResultStore}.
+     * @return CompletableFuture it the future with {@code true} if the dirty result is created
+     *     successfully, otherwise will throw {@link IllegalStateException} if the passed {@code
+     *     jobResultEntry} has a {@code JobID} attached that is already registered in this {@code
+     *     JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
-     * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
-     *     failed for IO reasons.
-     * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
+     * @return CompletableFuture is the future with the completed state, which will throw {@link
+     *     NoSuchElementException} if there is no corresponding {@code dirty} job present in the
      *     store for the given {@code JobID}.
      */
-    void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException;
+    CompletableFuture<Void> markResultAsCleanAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains an entry for a job.
+     * Returns the future of whether the store already contains an entry for a job.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for
-     *     the given {@code JobID}; otherwise {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return CompletableFuture with {@code true} if a {@code dirty} or {@code clean} {@link
+     *     JobResultEntry} exists for the given {@code JobID}; otherwise {@code false}.
      */
-    default boolean hasJobResultEntry(JobID jobId) throws IOException {
-        return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId);
+    default CompletableFuture<Boolean> hasJobResultEntryAsync(JobID jobId) {
+        return hasDirtyJobResultEntryAsync(jobId)
+                .thenCombine(
+                        hasCleanJobResultEntryAsync(jobId),
+                        (result1, result2) -> result1 || result2);
     }
 
     /**
-     * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code dirty} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return CompletableFuture with value of {@code true}, if a {@code dirty} entry exists for the
+     *     given {@code JobID}; otherwise Completable with value of {@code false}.
      */
-    boolean hasDirtyJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(JobID jobId);
 
     /**
-     * Returns whether the store already contains a {@code clean} entry for the given {@code JobID}.
+     * Returns the future of whether the store contains a {@code clean} entry for the given {@code
+     * JobID}.
      *
      * @param jobId Ident of the job we wish to check the store for.
-     * @return {@code true}, if a {@code clean} entry exists for the given {@code JobID}; otherwise
-     *     {@code false}.
-     * @throws IOException if determining whether a job entry is present in the store failed for IO
-     *     reasons.
+     * @return CompletableFuture with value of {@code true}, if a {@code clean} entry exists for the
+     *     given {@code JobID}; otherwise Completable with value of {@code false}.
      */
-    boolean hasCleanJobResultEntry(JobID jobId) throws IOException;
+    CompletableFuture<Boolean> hasCleanJobResultEntryAsync(JobID jobId);
 
     /**
-     * Get the persisted {@link JobResult} instances that are marked as {@code dirty}. This is
-     * useful for recovery of finalization steps.
+     * Returns the future of persisted {@link JobResult} instances that are marked as {@code dirty}.
+     * This is useful for recovery of finalization steps.
      *
-     * @return A set of dirty {@code JobResults} from the store.
-     * @throws IOException if collecting the set of dirty results failed for IO reasons.
+     * @return CompletableFuture with value of a set of dirty {@code JobResults} from the store.
      */
-    Set<JobResult> getDirtyResults() throws IOException;
+    CompletableFuture<Set<JobResult>> getDirtyResultsAsync();

Review Comment:
   Currently there are two methods `getDirtyResults ` and `getDirtyResultsAsync `.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264363723


##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java:
##########
@@ -332,11 +332,16 @@ public void testJobBeingMarkedAsDirtyBeforeCleanup() throws Exception {
                                 TestingJobResultStore.builder()
                                         .withCreateDirtyResultConsumer(
                                                 ignoredJobResultEntry -> {
+                                                    CompletableFuture<Boolean> result =
+                                                            new CompletableFuture<>();
                                                     try {
                                                         markAsDirtyLatch.await();
                                                     } catch (InterruptedException e) {
-                                                        throw new RuntimeException(e);
+                                                        result.completeExceptionally(
+                                                                new RuntimeException(e));

Review Comment:
   We need the `CompletableFuture` is completed with Exception?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1264612108


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -575,12 +576,13 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException {
      */
     private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException {
         try {
-            return jobResultStore.hasJobResultEntry(jobId);
-        } catch (IOException e) {
-            throw new FlinkException(
-                    String.format("Failed to retrieve job scheduling status for job %s.", jobId),
-                    e);
+            return jobResultStore.hasJobResultEntryAsync(jobId).get();

Review Comment:
   I've changed the return type of isInGloballyTerminalState.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1636989057

   Thanks for your careful review 😄 @XComp. Sorry for the first round of pull request because it's a bit rough. Please take a look when you have time.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] XComp commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1269591869


##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,87 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Boolean> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {

Review Comment:
   you could fix this in a hotfix commit in a generic fashion :shrug: 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1207840908


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1257,10 +1259,10 @@ private CompletableFuture<Void> removeJob(JobID jobId, CleanupJobState cleanupJo
 
     private void markJobAsClean(JobID jobId) {
         try {
-            jobResultStore.markResultAsClean(jobId);
+            jobResultStore.markResultAsCleanAsync(jobId).get();

Review Comment:
   OK. Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -1361,28 +1363,35 @@ private CompletableFuture<CleanupJobState> registerGloballyTerminatedJobInJobRes
                 "Job %s is in state %s which is not globally terminal.",
                 jobId,
                 terminalJobStatus);
-
-        ioExecutor.execute(
-                () -> {
-                    try {
-                        if (jobResultStore.hasCleanJobResultEntry(jobId)) {
-                            log.warn(
-                                    "Job {} is already marked as clean but clean up was triggered again.",
-                                    jobId);
-                        } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) {
-                            jobResultStore.createDirtyResult(
-                                    new JobResultEntry(
-                                            JobResult.createFrom(archivedExecutionGraph)));
-                            log.info(
-                                    "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.",
-                                    jobId);
-                        }
-                    } catch (IOException e) {
-                        writeFuture.completeExceptionally(e);
-                        return;
-                    }
-                    writeFuture.complete(null);
-                });
+        jobResultStore
+                .hasCleanJobResultEntryAsync(jobId)
+                .handleAsync(

Review Comment:
   OK. I have refactored these codes to the `CompletableFuture` style.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java:
##########
@@ -44,64 +45,84 @@ public abstract class AbstractThreadsafeJobResultStore implements JobResultStore
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
     @Override
-    public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException {
-        Preconditions.checkState(
-                !hasJobResultEntry(jobResultEntry.getJobId()),
-                "Job result store already contains an entry for job %s",
-                jobResultEntry.getJobId());
-
-        withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+    public CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry) {
+        return hasJobResultEntryAsync(jobResultEntry.getJobId())
+                .handle(
+                        (hasResult, error) -> {
+                            if (error != null || hasResult) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            try {
+                                withWriteLock(() -> createDirtyResultInternal(jobResultEntry));
+                            } catch (IOException e) {
+                                ExceptionUtils.rethrow(error);
+                            }
+                            return null;
+                        });
     }
 
     @GuardedBy("readWriteLock")
     protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry)
             throws IOException;
 
     @Override
-    public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException {
-        if (hasCleanJobResultEntry(jobId)) {
-            LOG.debug("The job {} is already marked as clean. No action required.", jobId);

Review Comment:
   The debug log is added.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,69 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
+     * @return CompletableFuture with the completed state.
      * @throws IOException if the creation of the dirty result failed for IO reasons.
      * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
      *     attached that is already registered in this {@code JobResultStore}.

Review Comment:
   Fixed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java:
##########
@@ -43,69 +44,69 @@ public interface JobResultStore {
      * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that
      * clean-up operations still need to be performed. Once the job resource cleanup has been
      * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link
-     * #markResultAsClean(JobID)}.
+     * #markResultAsCleanAsync(JobID)}.
      *
      * @param jobResultEntry The job result we wish to persist.
+     * @return CompletableFuture with the completed state.
      * @throws IOException if the creation of the dirty result failed for IO reasons.
      * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID}
      *     attached that is already registered in this {@code JobResultStore}.
      */
-    void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException;
+    CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry);
 
     /**
      * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more
      * resource cleanup steps need to be performed. No actions should be triggered if the passed
      * {@code JobID} belongs to a job that was already marked as clean.
      *
      * @param jobId Ident of the job we wish to mark as clean.
+     * @return CompletableFuture with the completed state.
      * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean}
      *     failed for IO reasons.
      * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the
      *     store for the given {@code JobID}.

Review Comment:
   Fixed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1207850160


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java:
##########
@@ -103,8 +103,8 @@ public static JobDispatcherLeaderProcessFactoryFactory create(
 
     private static Collection<JobResult> getDirtyJobResults(JobResultStore jobResultStore) {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   OK. Fixed. It makes sense that the method should propagate the CompletableFuture for readability purposes. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on a diff in pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on code in PR #22341:
URL: https://github.com/apache/flink/pull/22341#discussion_r1207855023


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -191,8 +191,8 @@ private Collection<JobResult> getDirtyJobResultsIfRunning() {
 
     private Collection<JobResult> getDirtyJobResults() {
         try {
-            return jobResultStore.getDirtyResults();
-        } catch (IOException e) {
+            return jobResultStore.getDirtyResultsAsync().get();

Review Comment:
   Done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] WencongLiu commented on pull request #22341: [FLINK-27204] Refract FileSystemJobResultStore to execute I/O operations on the ioExecutor

Posted by "WencongLiu (via GitHub)" <gi...@apache.org>.
WencongLiu commented on PR #22341:
URL: https://github.com/apache/flink/pull/22341#issuecomment-1505221567

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org