yigress commented on code in PR #23425:
URL: https://github.com/apache/flink/pull/23425#discussion_r1367568947


##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java:
##########
@@ -71,10 +81,19 @@ public void cleanCheckpoint(
             boolean shouldDiscard,
             Runnable postCleanAction,
             Executor executor) {
-        Checkpoint.DiscardObject discardObject =
-                shouldDiscard ? checkpoint.markAsDiscarded() : 
Checkpoint.NOOP_DISCARD_OBJECT;
-
-        cleanup(checkpoint, discardObject::discard, postCleanAction, executor);
+        LOG.debug(
+                "Clean checkpoint {} parallel-mode={} shouldDiscard={}",
+                checkpoint.getCheckpointID(),
+                parallelMode,
+                shouldDiscard);
+        if (parallelMode) {
+            cleanupParallel(checkpoint, shouldDiscard, postCleanAction, 
executor);
+        } else {
+            Checkpoint.DiscardObject discardObject =
+                    shouldDiscard ? checkpoint.markAsDiscarded() : 
Checkpoint.NOOP_DISCARD_OBJECT;
+
+            cleanup(checkpoint, discardObject::discard, postCleanAction, 
executor);

Review Comment:
   looks like DefaultCompletedCheckpointStore#shutdown calls the 
cleanCheckpoint from main thread instead of the ioExecutor. Combined into one 
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-unsubscr...@flink.apache.org

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

Reply via email to