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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java:
##########
@@ -71,10 +70,26 @@ 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);
+        if (shouldDiscard) {
+            incrementNumberOfCheckpointsToClean();
+            checkpoint
+                    .markAsDiscarded()
+                    .discardAsync(executor)
+                    .handle(
+                            (Object outerIgnored, Throwable outerThrowable) -> 
{
+                                if (outerThrowable != null) {
+                                    LOG.warn(
+                                            "Could not properly discard 
completed checkpoint {}.",
+                                            checkpoint.getCheckpointID(),
+                                            outerThrowable);
+                                }
+                                decrementNumberOfCheckpointsToClean();

Review Comment:
   `decrementNumberOfCheckpointsToClean` this should be also called if 
`shouldDiscard == false`, right?



##########
flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java:
##########
@@ -109,6 +109,20 @@ public class CheckpointingOptions {
                     .defaultValue(1)
                     .withDescription("The maximum number of completed 
checkpoints to retain.");
 
+    /**
+     * Option whether to clean individual checkpoint's operatorstates in 
parallel. If enabled,
+     * operator states are discarded in parallel using the ExecutorService 
passed to the cleaner.
+     * This speeds up checkpoints cleaning, but adds load to the IO.
+     */
+    @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+    public static final ConfigOption<Boolean> CLEANER_PARALLEL_MODE =
+            ConfigOptions.key("state.checkpoint.cleaner.parallel-mode")
+                    .booleanType()
+                    .defaultValue(false)

Review Comment:
   Ahh, sorry for missleading. I meant to keep the config option, but make the 
default value true, as this seems to be universally positive change, unless 
there is a bug/some unforeseen complication. 



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