aljoscha commented on a change in pull request #14734:
URL: https://github.com/apache/flink/pull/14734#discussion_r566172768



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
##########
@@ -183,23 +166,27 @@ public CheckpointStatsSnapshot createSnapshot() {
     /**
      * Creates a new pending checkpoint tracker.
      *
+     * @param tasksToAck Tasks to acknowledge in this checkpoint.
      * @param checkpointId ID of the checkpoint.
      * @param triggerTimestamp Trigger timestamp of the checkpoint.
      * @param props The checkpoint properties.
      * @return Tracker for statistics gathering.
      */
     PendingCheckpointStats reportPendingCheckpoint(
-            long checkpointId, long triggerTimestamp, CheckpointProperties 
props) {
+            List<ExecutionVertex> tasksToAck,

Review comment:
       Should still be `tasksToWaitFor`

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -825,7 +811,7 @@ private void snapshotTaskState(
             long checkpointID,
             CheckpointStorageLocation checkpointStorageLocation,
             CheckpointProperties props,
-            Execution[] executions,
+            List<Execution> executions,

Review comment:
       Here, a renaming might be helpful. This should really be 
`tasksToTrigger`, right?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
##########
@@ -90,6 +90,8 @@
 
     private final Map<ExecutionAttemptID, ExecutionVertex> 
notYetAcknowledgedTasks;
 
+    private final List<ExecutionVertex> runningTasks;

Review comment:
       Consistent with my other comment, these should still be named 
`tasksToCommitTo` for now.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -730,7 +715,8 @@ private PendingCheckpoint createPendingCheckpoint(
 
         if (statsTracker != null) {
             PendingCheckpointStats callback =
-                    statsTracker.reportPendingCheckpoint(checkpointID, 
timestamp, props);
+                    statsTracker.reportPendingCheckpoint(

Review comment:
       Should we maybe use `tasksToWaitFor` here instead of `tasksToCommit`? I 
think in the future the two numbers might differ but `tasksToCommitTo` will 
strictly have to be a subset of `tasksToWaitFor` because it might be that not 
all participating tasks need the "notify-complete".

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBrief.java
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The brief of one checkpoint, indicating which tasks to trigger, waiting for 
acknowledge or commit
+ * for one specific checkpoint.
+ */
+public class CheckpointBrief {
+
+    /** Tasks who need to be sent a message when a checkpoint is started. */
+    private final List<Execution> tasksToTrigger;
+
+    /** Tasks who need to acknowledge a checkpoint before it succeeds. */
+    private final Map<ExecutionAttemptID, ExecutionVertex> tasksToAck;
+
+    /**
+     * Tasks that are still running when taking the checkpoint, these need to 
be sent a message when
+     * the checkpoint is confirmed.
+     */
+    private final List<ExecutionVertex> runningTasks;

Review comment:
       Agreed, I think we should keep `tasksToTrigger`, `tasksToWaitFor`, and 
`tasksToCommitTo` for now. We can change the names in a pure renaming commit 
after that if we think the names are not good anymore.
   
   (I think the names could be clearer so it's actually good to change them... 
😅 )




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to