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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefCalculator.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.api.common.JobID;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Computes the tasks to trigger, wait or commit for each checkpoint. */
+public class CheckpointBriefCalculator {
+    private static final Logger LOG = 
LoggerFactory.getLogger(CheckpointBriefCalculator.class);
+
+    private final JobID jobId;
+
+    private final List<ExecutionVertex> tasksToTrigger;
+
+    private final List<ExecutionVertex> tasksToWait;
+
+    private final List<ExecutionVertex> tasksToCommitTo;
+
+    public CheckpointBriefCalculator(
+            JobID jobId,
+            List<ExecutionVertex> tasksToTrigger,
+            List<ExecutionVertex> tasksToWait,
+            List<ExecutionVertex> tasksToCommitTo) {
+
+        this.jobId = jobId;
+        this.tasksToTrigger = Collections.unmodifiableList(tasksToTrigger);
+        this.tasksToWait = Collections.unmodifiableList(tasksToWait);
+        this.tasksToCommitTo = Collections.unmodifiableList(tasksToCommitTo);
+    }
+
+    public CheckpointBrief calculateCheckpointBrief() throws 
CheckpointException {
+        List<Execution> tasksToTrigger = getTriggerExecutions();
+        Map<ExecutionAttemptID, ExecutionVertex> tasksToWait = getAckTasks();
+
+        return new CheckpointBrief(
+                Collections.unmodifiableList(tasksToTrigger), tasksToWait, 
tasksToCommitTo);

Review comment:
       Maybe the `tasksToWait` could also be `unmodifialbeMap`

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -2122,7 +2065,7 @@ public boolean isForce() {
 
     private Optional<ExecutionVertex> getVertex(ExecutionAttemptID id) throws 
CheckpointException {

Review comment:
       no `CheckpointException` any more.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -563,7 +554,7 @@ private void 
startTriggeringCheckpoint(CheckpointTriggerRequest request) {
                                             createPendingCheckpoint(
                                                     timestamp,
                                                     request.props,
-                                                    ackTasks,
+                                                    checkpointBrief,

Review comment:
       What I understand is that the `CheckpointBrief` describes how to “get” a 
`CompleteChekcpoint`. There are some methods related to“get”a 
CompleteCheckpoint, such as 
   
“createPendingCheckpoint/reportToStateTracker/sendAbortedMessage/sendAcknowledgeMessages”.
   
   However, these methods sometime use `CheckpointBrief`, sometime uses 
`tasksTowaitFor/tasksToTriggerFor`, which are the internal state of the 
`CheckpointCoordinator`. After we refactor maybe all the methods could only use 
the `CheckpointBrief` as their parameter. (I think CheckpointId,TimeStamps 
should also be in the CheckpointBrief).

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -1843,6 +1843,7 @@ int getNumQueuedRequests() {
 
     public void reportStats(long id, ExecutionAttemptID attemptId, 
CheckpointMetrics metrics)
             throws CheckpointException {
+

Review comment:
       maybe we could remove this line




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