rkhachatryan commented on a change in pull request #13040:
URL: https://github.com/apache/flink/pull/13040#discussion_r468849286



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -0,0 +1,86 @@
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Delegate class responsible for checkpoints cleaning and counting the number 
of checkpoints yet
+ * to clean.
+ */
+public class CheckpointsCleaner {
+       /** The executor used for asynchronous calls, like potentially blocking 
I/O. */
+       private final Executor executor;
+       AtomicInteger numberOfCheckpointsToClean;
+
+       Runnable checkpointCleaningFinishedCallback;
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(CheckpointsCleaner.class);
+
+       public CheckpointsCleaner(Executor executor) {
+               this.executor = executor;
+               this.numberOfCheckpointsToClean = new AtomicInteger(0);
+       }
+
+       public void setCheckpointCleaningFinishedCallback(Runnable 
checkpointCleaningFinishedCallback) {
+               this.checkpointCleaningFinishedCallback = 
checkpointCleaningFinishedCallback;
+       }
+
+       public int getNumberOfCheckpointsToClean() {
+               return numberOfCheckpointsToClean.get();
+       }
+
+       /**
+        * Asynchronously call a discard callback on on the ioExecutor
+        * (FixedThreadPool of configurable size of default 4*CPU cores)
+        * and count the number of checkpoints that are waiting to clean.
+        * @param completedCheckpoint the checkpoint to discard
+        * @param discardCallback the discard callback to call
+        */
+       public void 
asyncDiscardCheckpointAndCountCheckpoints(CompletedCheckpoint 
completedCheckpoint, ThrowingConsumer<CompletedCheckpoint, Exception> 
discardCallback){
+               numberOfCheckpointsToClean.incrementAndGet();
+               executor.execute(() -> {
+                       try {
+                               discardCallback.accept(completedCheckpoint);
+                       } catch (Exception e) {
+                               LOG.warn("Could not discard completed 
checkpoint {}.", completedCheckpoint.getCheckpointID(), e);
+                       }
+                       finally {
+                               numberOfCheckpointsToClean.decrementAndGet();
+                               
Preconditions.checkNotNull(checkpointCleaningFinishedCallback);
+                               checkpointCleaningFinishedCallback.run();
+                       }
+               });
+       }
+
+       public void asyncDiscardPrivateStatesAndCountCheckpoints(
+               Map<OperatorID, OperatorState> operatorStates, 
CheckpointStorageLocation targetLocation,
+               long checkpointId, JobID jobId) {
+               numberOfCheckpointsToClean.incrementAndGet();
+               executor.execute(() -> {
+                       // discard the private states.
+                       // unregistered shared states are still considered 
private at this point.
+                       try {
+                               
StateUtil.bestEffortDiscardAllStateObjects(operatorStates.values());
+                               targetLocation.disposeOnFailure();
+                       } catch (Throwable t) {
+                               LOG.warn(
+                                       "Could not properly dispose the private 
states in the pending checkpoint {} of job {}.",
+                                       checkpointId, jobId, t);
+                       } finally {
+                               operatorStates.clear();
+                               numberOfCheckpointsToClean.decrementAndGet();

Review comment:
       Should `checkpointCleaningFinishedCallback.run()` be here too?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -0,0 +1,86 @@
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Delegate class responsible for checkpoints cleaning and counting the number 
of checkpoints yet
+ * to clean.
+ */
+public class CheckpointsCleaner {

Review comment:
       Please reduce the visibility of this class, its methods, and its fields 
as much as possible.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -0,0 +1,86 @@
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Delegate class responsible for checkpoints cleaning and counting the number 
of checkpoints yet
+ * to clean.
+ */
+public class CheckpointsCleaner {
+       /** The executor used for asynchronous calls, like potentially blocking 
I/O. */
+       private final Executor executor;
+       AtomicInteger numberOfCheckpointsToClean;
+
+       Runnable checkpointCleaningFinishedCallback;
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(CheckpointsCleaner.class);
+
+       public CheckpointsCleaner(Executor executor) {
+               this.executor = executor;
+               this.numberOfCheckpointsToClean = new AtomicInteger(0);
+       }
+
+       public void setCheckpointCleaningFinishedCallback(Runnable 
checkpointCleaningFinishedCallback) {
+               this.checkpointCleaningFinishedCallback = 
checkpointCleaningFinishedCallback;
+       }
+
+       public int getNumberOfCheckpointsToClean() {
+               return numberOfCheckpointsToClean.get();
+       }
+
+       /**
+        * Asynchronously call a discard callback on on the ioExecutor
+        * (FixedThreadPool of configurable size of default 4*CPU cores)
+        * and count the number of checkpoints that are waiting to clean.
+        * @param completedCheckpoint the checkpoint to discard
+        * @param discardCallback the discard callback to call
+        */
+       public void 
asyncDiscardCheckpointAndCountCheckpoints(CompletedCheckpoint 
completedCheckpoint, ThrowingConsumer<CompletedCheckpoint, Exception> 
discardCallback){
+               numberOfCheckpointsToClean.incrementAndGet();
+               executor.execute(() -> {
+                       try {
+                               discardCallback.accept(completedCheckpoint);
+                       } catch (Exception e) {
+                               LOG.warn("Could not discard completed 
checkpoint {}.", completedCheckpoint.getCheckpointID(), e);
+                       }
+                       finally {
+                               numberOfCheckpointsToClean.decrementAndGet();
+                               
Preconditions.checkNotNull(checkpointCleaningFinishedCallback);
+                               checkpointCleaningFinishedCallback.run();
+                       }
+               });
+       }
+
+       public void asyncDiscardPrivateStatesAndCountCheckpoints(
+               Map<OperatorID, OperatorState> operatorStates, 
CheckpointStorageLocation targetLocation,
+               long checkpointId, JobID jobId) {
+               numberOfCheckpointsToClean.incrementAndGet();

Review comment:
       ditto
   Should this call be inside the same `try/finally` as decrement?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -0,0 +1,86 @@
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Delegate class responsible for checkpoints cleaning and counting the number 
of checkpoints yet
+ * to clean.
+ */
+public class CheckpointsCleaner {
+       /** The executor used for asynchronous calls, like potentially blocking 
I/O. */
+       private final Executor executor;
+       AtomicInteger numberOfCheckpointsToClean;
+
+       Runnable checkpointCleaningFinishedCallback;

Review comment:
       This field should be `volatile` (in addition to `private`).

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -0,0 +1,86 @@
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Delegate class responsible for checkpoints cleaning and counting the number 
of checkpoints yet
+ * to clean.
+ */
+public class CheckpointsCleaner {
+       /** The executor used for asynchronous calls, like potentially blocking 
I/O. */
+       private final Executor executor;
+       AtomicInteger numberOfCheckpointsToClean;

Review comment:
       This field should be `final` (in addition to `private`).

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
##########
@@ -0,0 +1,86 @@
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Delegate class responsible for checkpoints cleaning and counting the number 
of checkpoints yet
+ * to clean.
+ */
+public class CheckpointsCleaner {
+       /** The executor used for asynchronous calls, like potentially blocking 
I/O. */
+       private final Executor executor;
+       AtomicInteger numberOfCheckpointsToClean;
+
+       Runnable checkpointCleaningFinishedCallback;
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(CheckpointsCleaner.class);
+
+       public CheckpointsCleaner(Executor executor) {
+               this.executor = executor;
+               this.numberOfCheckpointsToClean = new AtomicInteger(0);
+       }
+
+       public void setCheckpointCleaningFinishedCallback(Runnable 
checkpointCleaningFinishedCallback) {
+               this.checkpointCleaningFinishedCallback = 
checkpointCleaningFinishedCallback;
+       }
+
+       public int getNumberOfCheckpointsToClean() {
+               return numberOfCheckpointsToClean.get();
+       }
+
+       /**
+        * Asynchronously call a discard callback on on the ioExecutor
+        * (FixedThreadPool of configurable size of default 4*CPU cores)
+        * and count the number of checkpoints that are waiting to clean.
+        * @param completedCheckpoint the checkpoint to discard
+        * @param discardCallback the discard callback to call
+        */
+       public void 
asyncDiscardCheckpointAndCountCheckpoints(CompletedCheckpoint 
completedCheckpoint, ThrowingConsumer<CompletedCheckpoint, Exception> 
discardCallback){
+               numberOfCheckpointsToClean.incrementAndGet();

Review comment:
       Should this call be inside the same `try/finally` as decrement?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java
##########
@@ -459,7 +460,8 @@ private CheckpointCoordinator 
instantiateCheckpointCoordinator(
                                new StandaloneCheckpointIDCounter(),
                                new StandaloneCompletedCheckpointStore(10),
                                new MemoryStateBackend(),
-                               Executors.directExecutor(),
+                               executor,
+                               new CheckpointsCleaner(executor),

Review comment:
       How about creating an interface (`CheckpointCleaner`) and a no-op 
implementation and use it in tests where possible?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
##########
@@ -519,24 +522,8 @@ private void dispose(boolean releaseState) {
                        try {
                                numAcknowledgedTasks = -1;
                                if (!discarded && releaseState) {
-                                       executor.execute(new Runnable() {
-                                               @Override
-                                               public void run() {
-
-                                                       // discard the private 
states.
-                                                       // unregistered shared 
states are still considered private at this point.
-                                                       try {
-                                                               
StateUtil.bestEffortDiscardAllStateObjects(operatorStates.values());
-                                                               
targetLocation.disposeOnFailure();
-                                                       } catch (Throwable t) {
-                                                               LOG.warn("Could 
not properly dispose the private states in the pending checkpoint {} of job 
{}.",
-                                                                       
checkpointId, jobId, t);
-                                                       } finally {
-                                                               
operatorStates.clear();
-                                                       }
-                                               }
-                                       });
-
+                                       
checkpointsCleaner.asyncDiscardPrivateStatesAndCountCheckpoints(operatorStates,

Review comment:
       I think synchronization on `lock` is lost here (`checkpointsCleaner` 
uses other thread).
   To fix it, `checkpointsCleaner` could accept `pendingCheckpoint`, call some 
method on it, which internally synchronizes on `lock`.
   WDYT?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
##########
@@ -44,17 +44,17 @@ public ZooKeeperCheckpointRecoveryFactory(
                        CuratorFramework client,
                        Configuration config,
                        Executor executor) {
-               this.client = checkNotNull(client, "Curator client");
+                       this.client = checkNotNull(client, "Curator client");

Review comment:
       Broken indentation.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
##########
@@ -289,7 +290,7 @@ public static CompletedCheckpointStore 
createCompletedCheckpoints(
                        Configuration configuration,
                        JobID jobId,
                        int maxNumberOfCheckpointsToRetain,
-                       Executor executor) throws Exception {
+                       Executor executor, CheckpointsCleaner 
checkpointsCleaner) throws Exception {

Review comment:
       According to [Flink code style 
guide](https://flink.apache.org/contributing/code-style-and-quality-formatting.html)
   all arguments must either be on a single line, or all on separate lines.
   (please revisit all such places)




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

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


Reply via email to