gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570136359



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -223,16 +214,25 @@
     private boolean isTriggering = false;
 
     private final CheckpointRequestDecider requestDecider;
+
+    private final CheckpointBriefCalculator checkpointBriefCalculator;
+
+    private final ExecutionAttemptMappingProvider attemptMappingProvider;
+
     private final LinkedHashMap<ExecutionAttemptID, ExecutionVertex> 
cachedTasksById;
 
+    /**
+     * Temporary flag to allow checkpoints after tasks finished. This is 
disabled for regular jobs
+     * to keep the current behavior but we want to allow it in tests. This 
should be removed once
+     * all parts of the stack support checkpoints after some tasks finished.
+     */
+    private boolean allowCheckpointsAfterTasksFinished;

Review comment:
       Yes, this is not a feature toggle and would be removed in the final PR. 
   
   The flag (and also the flag in the TM side) is required since if not all of 
this PRs are get in some behaviors would be not right. We would remove all 
these flags in the last PR, which enables the whole functionality atomically.  

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefCalculator.java
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** 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 CheckpointBriefCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new 
ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public CheckpointBriefCalculator(
+            JobID jobId,
+            CheckpointBriefCalculatorContext context,
+            Iterable<ExecutionJobVertex> jobVerticesInTopologyOrderIterable) {
+
+        this.jobId = checkNotNull(jobId);
+        this.context = checkNotNull(context);
+
+        checkNotNull(jobVerticesInTopologyOrderIterable);
+        jobVerticesInTopologyOrderIterable.forEach(
+                jobVertex -> {
+                    jobVerticesInTopologyOrder.add(jobVertex);
+                    
allTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        
sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    public CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        CompletableFuture<CheckpointBrief> resultFuture = new 
CompletableFuture<>();
+
+        context.getMainExecutor()
+                .execute(
+                        () -> {

Review comment:
       I think it would be much simpler, very thanks!

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefCalculator.java
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** 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 CheckpointBriefCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new 
ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public CheckpointBriefCalculator(
+            JobID jobId,
+            CheckpointBriefCalculatorContext context,
+            Iterable<ExecutionJobVertex> jobVerticesInTopologyOrderIterable) {
+
+        this.jobId = checkNotNull(jobId);
+        this.context = checkNotNull(context);
+
+        checkNotNull(jobVerticesInTopologyOrderIterable);
+        jobVerticesInTopologyOrderIterable.forEach(
+                jobVertex -> {
+                    jobVerticesInTopologyOrder.add(jobVertex);
+                    
allTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        
sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    public CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        CompletableFuture<CheckpointBrief> resultFuture = new 
CompletableFuture<>();
+
+        context.getMainExecutor()
+                .execute(
+                        () -> {
+                            try {
+                                if (!isAllExecutionAttemptsAreInitiated()) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                CheckpointBrief result;
+                                if (!context.hasFinishedTasks()) {
+                                    result = calculateWithAllTasksRunning();
+                                } else {
+                                    result = calculateAfterTasksFinished();
+                                }
+
+                                if 
(!isAllExecutionsToTriggerStarted(result.getTasksToTrigger())) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                resultFuture.complete(result);
+                            } catch (Throwable throwable) {
+                                resultFuture.completeExceptionally(throwable);
+                            }
+                        });
+
+        return resultFuture;
+    }
+
+    private boolean isAllExecutionAttemptsAreInitiated() {
+        for (ExecutionVertex task : allTasks) {
+            if (task.getCurrentExecutionAttempt() == null) {
+                LOG.info(
+                        "task {} of job {} is not being executed at the 
moment. Aborting checkpoint.",
+                        task.getTaskNameWithSubtaskIndex(),
+                        jobId);
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private boolean isAllExecutionsToTriggerStarted(List<Execution> toTrigger) 
{
+        for (Execution execution : toTrigger) {
+            if (execution.getState() == ExecutionState.CREATED
+                    || execution.getState() == ExecutionState.SCHEDULED
+                    || execution.getState() == ExecutionState.DEPLOYING) {
+
+                LOG.info(
+                        "Checkpoint triggering task {} of job {} has not being 
executed at the moment. "
+                                + "Aborting checkpoint.",
+                        execution.getVertex().getTaskNameWithSubtaskIndex(),
+                        jobId);
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * Computes the checkpoint brief when all tasks are running. It would 
simply marks all the
+     * source tasks as need to trigger and all the tasks as need to wait and 
commit.
+     *
+     * @return The brief of this checkpoint.
+     */
+    private CheckpointBrief calculateWithAllTasksRunning() {
+        List<Execution> executionsToTrigger =
+                sourceTasks.stream()
+                        .map(ExecutionVertex::getCurrentExecutionAttempt)
+                        .collect(Collectors.toList());
+
+        Map<ExecutionAttemptID, ExecutionVertex> ackTasks = 
createTaskToAck(allTasks);
+
+        return new CheckpointBrief(
+                Collections.unmodifiableList(executionsToTrigger),
+                ackTasks,
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Computes the checkpoint brief after some tasks have finished. Due to 
the problem of the order
+     * of reporting FINISHED is nondeterministic, we have to first compute the 
accurate running
+     * tasks. Then we would iterate the job graph to find the task that is 
still running, but do not
+     * has precedent running tasks.
+     *
+     * @return The brief of this checkpoint.
+     */
+    private CheckpointBrief calculateAfterTasksFinished() {
+        Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex = 
calculateRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        Map<ExecutionAttemptID, ExecutionVertex> tasksToAck = new HashMap<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            JobVertexTaskSet runningTasks = 
runningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (runningTasks.containsNoTasks()) {
+                fullyFinishedJobVertex.add(jobVertex);
+            }

Review comment:
       I agree with it would be more natural, I updated this way.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefCalculator.java
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** 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 CheckpointBriefCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new 
ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public CheckpointBriefCalculator(
+            JobID jobId,
+            CheckpointBriefCalculatorContext context,
+            Iterable<ExecutionJobVertex> jobVerticesInTopologyOrderIterable) {
+
+        this.jobId = checkNotNull(jobId);
+        this.context = checkNotNull(context);
+
+        checkNotNull(jobVerticesInTopologyOrderIterable);
+        jobVerticesInTopologyOrderIterable.forEach(
+                jobVertex -> {
+                    jobVerticesInTopologyOrder.add(jobVertex);
+                    
allTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        
sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    public CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        CompletableFuture<CheckpointBrief> resultFuture = new 
CompletableFuture<>();
+
+        context.getMainExecutor()
+                .execute(
+                        () -> {
+                            try {
+                                if (!isAllExecutionAttemptsAreInitiated()) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                CheckpointBrief result;
+                                if (!context.hasFinishedTasks()) {
+                                    result = calculateWithAllTasksRunning();
+                                } else {
+                                    result = calculateAfterTasksFinished();
+                                }
+
+                                if 
(!isAllExecutionsToTriggerStarted(result.getTasksToTrigger())) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                resultFuture.complete(result);
+                            } catch (Throwable throwable) {
+                                resultFuture.completeExceptionally(throwable);
+                            }
+                        });
+
+        return resultFuture;
+    }
+
+    private boolean isAllExecutionAttemptsAreInitiated() {
+        for (ExecutionVertex task : allTasks) {
+            if (task.getCurrentExecutionAttempt() == null) {
+                LOG.info(
+                        "task {} of job {} is not being executed at the 
moment. Aborting checkpoint.",
+                        task.getTaskNameWithSubtaskIndex(),
+                        jobId);
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private boolean isAllExecutionsToTriggerStarted(List<Execution> toTrigger) 
{
+        for (Execution execution : toTrigger) {
+            if (execution.getState() == ExecutionState.CREATED
+                    || execution.getState() == ExecutionState.SCHEDULED
+                    || execution.getState() == ExecutionState.DEPLOYING) {
+
+                LOG.info(
+                        "Checkpoint triggering task {} of job {} has not being 
executed at the moment. "
+                                + "Aborting checkpoint.",
+                        execution.getVertex().getTaskNameWithSubtaskIndex(),
+                        jobId);
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * Computes the checkpoint brief when all tasks are running. It would 
simply marks all the
+     * source tasks as need to trigger and all the tasks as need to wait and 
commit.
+     *
+     * @return The brief of this checkpoint.
+     */
+    private CheckpointBrief calculateWithAllTasksRunning() {
+        List<Execution> executionsToTrigger =
+                sourceTasks.stream()
+                        .map(ExecutionVertex::getCurrentExecutionAttempt)
+                        .collect(Collectors.toList());
+
+        Map<ExecutionAttemptID, ExecutionVertex> ackTasks = 
createTaskToAck(allTasks);
+
+        return new CheckpointBrief(
+                Collections.unmodifiableList(executionsToTrigger),
+                ackTasks,
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Computes the checkpoint brief after some tasks have finished. Due to 
the problem of the order
+     * of reporting FINISHED is nondeterministic, we have to first compute the 
accurate running
+     * tasks. Then we would iterate the job graph to find the task that is 
still running, but do not
+     * has precedent running tasks.
+     *
+     * @return The brief of this checkpoint.
+     */
+    private CheckpointBrief calculateAfterTasksFinished() {
+        Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex = 
calculateRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        Map<ExecutionAttemptID, ExecutionVertex> tasksToAck = new HashMap<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            JobVertexTaskSet runningTasks = 
runningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (runningTasks.containsNoTasks()) {
+                fullyFinishedJobVertex.add(jobVertex);
+            }
+
+            List<JobEdge> prevJobEdges = jobVertex.getJobVertex().getInputs();
+
+            // this is an optimization: we determine at the JobVertex level if 
some tasks can even
+            // be eligible for being in the "triggerTo" set
+            boolean someTasksMustBeTriggered =
+                    someTasksMustBeTriggered(runningTasksByVertex, 
runningTasks, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (runningTasks.contains(vertex.getID())) {
+                    
tasksToAck.put(vertex.getCurrentExecutionAttempt().getAttemptId(), vertex);
+
+                    if (someTasksMustBeTriggered) {
+                        boolean hasRunningPrecedentTasks =
+                                hasRunningPrecedentTasks(
+                                        runningTasksByVertex, prevJobEdges, 
vertex);
+
+                        if (!hasRunningPrecedentTasks) {
+                            
tasksToTrigger.add(vertex.getCurrentExecutionAttempt());
+                        }
+                    }
+                } else {
+                    finishedTasks.add(vertex.getCurrentExecutionAttempt());
+                }
+            }
+        }
+
+        return new CheckpointBrief(
+                Collections.unmodifiableList(tasksToTrigger),
+                tasksToAck,
+                Collections.unmodifiableList(
+                        tasksToAck.size() == allTasks.size()
+                                ? allTasks
+                                : new ArrayList<>(tasksToAck.values())),
+                Collections.unmodifiableList(finishedTasks),
+                Collections.unmodifiableList(fullyFinishedJobVertex));
+    }
+
+    private boolean hasRunningPrecedentTasks(
+            Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex,
+            List<JobEdge> prevJobEdges,
+            ExecutionVertex vertex) {
+        return IntStream.range(0, prevJobEdges.size())
+                .filter(
+                        i ->
+                                prevJobEdges.get(i).getDistributionPattern()
+                                        == DistributionPattern.POINTWISE)
+                .boxed()
+                .flatMap(i -> getPrecedentTasks(vertex, i).stream())
+                .anyMatch(
+                        precedentTask ->
+                                runningTasksByVertex
+                                        .get(precedentTask.getJobvertexId())
+                                        .contains(precedentTask.getID()));
+    }
+
+    private boolean someTasksMustBeTriggered(
+            Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex,
+            JobVertexTaskSet runningTasks,
+            List<JobEdge> prevJobEdges) {
+        if (runningTasks.containsNoTasks()) {
+            // if this task doesn't have any running tasks, we cannot trigger 
anything
+            return false;
+        }
+        for (JobEdge jobEdge : prevJobEdges) {
+            DistributionPattern distributionPattern = 
jobEdge.getDistributionPattern();
+            JobVertexTaskSet sourceRunningTasks =
+                    
runningTasksByVertex.get(jobEdge.getSource().getProducer().getID());
+
+            if ((distributionPattern == DistributionPattern.ALL_TO_ALL
+                            && !sourceRunningTasks.containsNoTasks())
+                    || (distributionPattern == DistributionPattern.POINTWISE
+                            && sourceRunningTasks.containsAllTasks())) {

Review comment:
       I'm a bit concerns here in that the condition used for calculate trigger 
tasks and calculate finished tasks are a bit different:
   1. For calculating tasks to trigger, it would check if it is ALL_TO_ALL 
connected to some running tasks or POINTWISE connected to all running tasks.
   2. For judge if a task is finished, it would check if it is ALL_TO_ALL 
connected to some finished tasks (namely not all tasks are running) or 
POINTWISE connected to all finished tasks.
   
   Thus it seems to be not easy to extract a common method~?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefCalculator.java
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** 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 CheckpointBriefCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new 
ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public CheckpointBriefCalculator(
+            JobID jobId,
+            CheckpointBriefCalculatorContext context,
+            Iterable<ExecutionJobVertex> jobVerticesInTopologyOrderIterable) {
+
+        this.jobId = checkNotNull(jobId);
+        this.context = checkNotNull(context);
+
+        checkNotNull(jobVerticesInTopologyOrderIterable);
+        jobVerticesInTopologyOrderIterable.forEach(
+                jobVertex -> {
+                    jobVerticesInTopologyOrder.add(jobVertex);
+                    
allTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        
sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    public CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        CompletableFuture<CheckpointBrief> resultFuture = new 
CompletableFuture<>();
+
+        context.getMainExecutor()
+                .execute(
+                        () -> {
+                            try {
+                                if (!isAllExecutionAttemptsAreInitiated()) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                CheckpointBrief result;
+                                if (!context.hasFinishedTasks()) {
+                                    result = calculateWithAllTasksRunning();
+                                } else {
+                                    result = calculateAfterTasksFinished();
+                                }
+
+                                if 
(!isAllExecutionsToTriggerStarted(result.getTasksToTrigger())) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                resultFuture.complete(result);
+                            } catch (Throwable throwable) {
+                                resultFuture.completeExceptionally(throwable);
+                            }
+                        });
+
+        return resultFuture;
+    }
+
+    private boolean isAllExecutionAttemptsAreInitiated() {
+        for (ExecutionVertex task : allTasks) {
+            if (task.getCurrentExecutionAttempt() == null) {
+                LOG.info(
+                        "task {} of job {} is not being executed at the 
moment. Aborting checkpoint.",
+                        task.getTaskNameWithSubtaskIndex(),
+                        jobId);
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private boolean isAllExecutionsToTriggerStarted(List<Execution> toTrigger) 
{

Review comment:
       I think it would indeed be simpler to directly throw the exception 
inside the method and change the method names. I have updated in this way.
   
   But I'm a bit concerns in changing the log output, since it might be not 
consist with users' custom (namely from the log to the exception stack), do you 
think it would be ok for us to keep the log, or we add cause to the exception 
but still keep `log.info`~?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -2132,23 +2123,41 @@ public boolean isForce() {
     }
 
     private void reportToStatsTracker(
-            PendingCheckpoint checkpoint, Map<ExecutionAttemptID, 
ExecutionVertex> tasks) {
+            PendingCheckpoint checkpoint,
+            Map<ExecutionAttemptID, ExecutionVertex> tasks,
+            List<Execution> finishedTasks) {
         if (statsTracker == null) {
             return;
         }
         Map<JobVertexID, Integer> vertices =
-                tasks.values().stream()
+                Stream.concat(
+                                tasks.values().stream(),
+                                
finishedTasks.stream().map(Execution::getVertex))
                         .map(ExecutionVertex::getJobVertex)
                         .distinct()
                         .collect(
                                 toMap(
                                         ExecutionJobVertex::getJobVertexId,
                                         ExecutionJobVertex::getParallelism));
-        checkpoint.setStatsCallback(
+
+        PendingCheckpointStats pendingCheckpointStats =
                 statsTracker.reportPendingCheckpoint(
                         checkpoint.getCheckpointID(),
                         checkpoint.getCheckpointTimestamp(),
                         checkpoint.getProps(),
-                        vertices));
+                        vertices);
+        checkpoint.setStatsCallback(pendingCheckpointStats);
+
+        reportFinishedTasks(pendingCheckpointStats, finishedTasks);
+    }
+
+    private void reportFinishedTasks(
+            PendingCheckpointStats pendingCheckpointStats, List<Execution> 
finishedTasks) {
+        long now = System.currentTimeMillis();
+        finishedTasks.forEach(
+                execution ->
+                        pendingCheckpointStats.reportSubtaskStats(

Review comment:
       Yes, currently it would report 0 for the metrics of finished tasks. 
   
   I think it would be desired since if we do not report these tasks, users 
would be not easy to know which tasks are finished when the checkpoint trigger, 
thus he could not easily distinguish the finished tasks with the tasks that 
indeed not report snapshot for some reason. We may also consider add another 
flag to indicate if a task is finished when triggering checkpoints in a 
separate issue. 

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefCalculator.java
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** 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 CheckpointBriefCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new 
ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public CheckpointBriefCalculator(
+            JobID jobId,
+            CheckpointBriefCalculatorContext context,
+            Iterable<ExecutionJobVertex> jobVerticesInTopologyOrderIterable) {
+
+        this.jobId = checkNotNull(jobId);
+        this.context = checkNotNull(context);
+
+        checkNotNull(jobVerticesInTopologyOrderIterable);
+        jobVerticesInTopologyOrderIterable.forEach(
+                jobVertex -> {
+                    jobVerticesInTopologyOrder.add(jobVertex);
+                    
allTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        
sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    public CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        CompletableFuture<CheckpointBrief> resultFuture = new 
CompletableFuture<>();
+
+        context.getMainExecutor()
+                .execute(
+                        () -> {
+                            try {
+                                if (!isAllExecutionAttemptsAreInitiated()) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                CheckpointBrief result;
+                                if (!context.hasFinishedTasks()) {
+                                    result = calculateWithAllTasksRunning();
+                                } else {
+                                    result = calculateAfterTasksFinished();
+                                }
+
+                                if 
(!isAllExecutionsToTriggerStarted(result.getTasksToTrigger())) {
+                                    throw new CheckpointException(
+                                            
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                                }
+
+                                resultFuture.complete(result);
+                            } catch (Throwable throwable) {
+                                resultFuture.completeExceptionally(throwable);
+                            }
+                        });
+
+        return resultFuture;
+    }
+
+    private boolean isAllExecutionAttemptsAreInitiated() {
+        for (ExecutionVertex task : allTasks) {
+            if (task.getCurrentExecutionAttempt() == null) {

Review comment:
       I add comments to this method about it must be called in main JM thread. 
I think currently comments would be ok since this method is private and this 
class could ensures the calculation is done inside the main JM thread.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
##########
@@ -611,6 +590,10 @@ public long getNumberOfRestarts() {
         return numberOfRestartsCounter.getCount();
     }
 
+    public int getVerticesFinished() {

Review comment:
       I also think `getFinishedVertices` would be more nature, but a bit 
concern here is that the variable to get is name by `verticesFinished`, should 
we keeps this method to be a getter method for that variable ?




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