You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/27 08:45:43 UTC

[GitHub] [flink] zentol commented on a change in pull request #17572: [FLINK-23647][checkpointing] Wait for CheckpointsCleaner to complete before closing cluster

zentol commented on a change in pull request #17572:
URL: https://github.com/apache/flink/pull/17572#discussion_r737238906



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -89,9 +95,34 @@ private void cleanup(
                                 checkpoint.getCheckpointID(),
                                 e);
                     } finally {
-                        numberOfCheckpointsToClean.decrementAndGet();
+                        decrementNumberOfCheckpointsToClean();
                         postCleanupAction.run();
                     }
                 });
     }
+
+    private synchronized void decrementNumberOfCheckpointsToClean() {
+        numberOfCheckpointsToClean--;
+        maybeCompleteCloseUnsafe();
+    }
+
+    private void maybeCompleteCloseUnsafe() {
+        if (numberOfCheckpointsToClean == 0 && cleanUpFuture != null) {
+            cleanUpFuture.complete(null);
+        }
+    }
+
+    private synchronized void incrementNumberOfCheckpointsToClean() {
+        checkState(cleanUpFuture == null, "CheckpointsCleaner has already been closed");
+        numberOfCheckpointsToClean++;
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> closeAsync() {
+        if (cleanUpFuture == null) {
+            cleanUpFuture = new CompletableFuture<>();

Review comment:
       It seems a bit sketchy for the Dispatcher to wait for the cleanup although it hasn't directly initiated it. Who kicks of the actual cleanup? Is it the CheckpointCoordinator? In that case, why don't we have the coordinator shutdown wait for the cleanup?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -89,9 +95,34 @@ private void cleanup(
                                 checkpoint.getCheckpointID(),
                                 e);
                     } finally {
-                        numberOfCheckpointsToClean.decrementAndGet();
+                        decrementNumberOfCheckpointsToClean();
                         postCleanupAction.run();
                     }
                 });
     }
+
+    private synchronized void decrementNumberOfCheckpointsToClean() {
+        numberOfCheckpointsToClean--;
+        maybeCompleteCloseUnsafe();
+    }
+
+    private void maybeCompleteCloseUnsafe() {
+        if (numberOfCheckpointsToClean == 0 && cleanUpFuture != null) {
+            cleanUpFuture.complete(null);
+        }
+    }
+
+    private synchronized void incrementNumberOfCheckpointsToClean() {
+        checkState(cleanUpFuture == null, "CheckpointsCleaner has already been closed");
+        numberOfCheckpointsToClean++;
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> closeAsync() {

Review comment:
       Using Closeable is misleading because I would expect this call to initiate the closing, not just wait for some other component to trigger it in the background.




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