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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to