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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointPlanCalculator.java
##########
@@ -18,109 +18,498 @@
 
 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.concurrent.CompletionException;
+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 CheckpointPlanCalculator {
     private static final Logger LOG = 
LoggerFactory.getLogger(CheckpointPlanCalculator.class);
 
     private final JobID jobId;
 
-    private final List<ExecutionVertex> tasksToTrigger;
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new 
ArrayList<>();
 
-    private final List<ExecutionVertex> tasksToWait;
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
 
-    private final List<ExecutionVertex> tasksToCommitTo;
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
 
     public CheckpointPlanCalculator(
             JobID jobId,
-            List<ExecutionVertex> tasksToTrigger,
-            List<ExecutionVertex> tasksToWait,
-            List<ExecutionVertex> tasksToCommitTo) {
+            CheckpointPlanCalculatorContext 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()));
 
-        this.jobId = jobId;
-        this.tasksToTrigger = Collections.unmodifiableList(tasksToTrigger);
-        this.tasksToWait = Collections.unmodifiableList(tasksToWait);
-        this.tasksToCommitTo = Collections.unmodifiableList(tasksToCommitTo);
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        
sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
     }
 
-    public CheckpointPlan calculateCheckpointPlan() throws CheckpointException 
{
-        return new CheckpointPlan(
-                Collections.unmodifiableList(getTriggerExecutions()),
-                Collections.unmodifiableMap(getAckTasks()),
-                tasksToCommitTo);
+    public CompletableFuture<CheckpointPlan> calculateCheckpointPlan() {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        checkAllTasksInitiated();
+
+                        CheckpointPlan result =
+                                context.hasFinishedTasks()
+                                        ? calculateAfterTasksFinished()
+                                        : calculateWithAllTasksRunning();
+
+                        checkTasksStarted(result.getTasksToTrigger());
+
+                        return result;
+                    } catch (Throwable throwable) {
+                        throw new CompletionException(throwable);
+                    }
+                },
+                context.getMainExecutor());
     }
 
     /**
-     * Check if all tasks that we need to trigger are running. If not, abort 
the checkpoint.
+     * Checks if all tasks are attached with the current Execution already. 
This method should be
+     * called from JobMaster main thread executor.
      *
-     * @return the executions need to be triggered.
-     * @throws CheckpointException the exception fails checking
+     * @throws CheckpointException if some tasks do not have attached 
Execution.
      */
-    private List<Execution> getTriggerExecutions() throws CheckpointException {
-        List<Execution> executionsToTrigger = new 
ArrayList<>(tasksToTrigger.size());
-        for (ExecutionVertex executionVertex : tasksToTrigger) {
-            Execution ee = executionVertex.getCurrentExecutionAttempt();
-            if (ee == null) {
-                LOG.info(
-                        "Checkpoint triggering task {} of job {} is not being 
executed at the moment. Aborting checkpoint.",
-                        executionVertex.getTaskNameWithSubtaskIndex(),
-                        executionVertex.getJobId());
+    private void checkAllTasksInitiated() throws CheckpointException {
+        for (ExecutionVertex task : allTasks) {
+            if (task.getCurrentExecutionAttempt() == null) {
                 throw new CheckpointException(
+                        String.format(
+                                "task %s of job %s is not being executed at 
the moment. Aborting checkpoint.",
+                                task.getTaskNameWithSubtaskIndex(), jobId),
                         
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
-            } else if (ee.getState() == ExecutionState.RUNNING) {
-                executionsToTrigger.add(ee);
-            } else {
-                LOG.info(
-                        "Checkpoint triggering task {} of job {} is not in 
state {} but {} instead. Aborting checkpoint.",
-                        executionVertex.getTaskNameWithSubtaskIndex(),
-                        jobId,
-                        ExecutionState.RUNNING,
-                        ee.getState());
+            }
+        }
+    }
+
+    /**
+     * Checks if all tasks to trigger have already been in RUNNING state. This 
method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks to trigger have not turned 
into RUNNING yet.
+     */
+    private void checkTasksStarted(List<Execution> toTrigger) throws 
CheckpointException {
+        for (Execution execution : toTrigger) {
+            if (execution.getState() == ExecutionState.CREATED
+                    || execution.getState() == ExecutionState.SCHEDULED
+                    || execution.getState() == ExecutionState.DEPLOYING) {
+
                 throw new CheckpointException(
+                        String.format(
+                                "Checkpoint triggering task %s of job %s has 
not being executed at the moment. "
+                                        + "Aborting checkpoint.",
+                                
execution.getVertex().getTaskNameWithSubtaskIndex(), jobId),
                         
CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
             }
         }
+    }
+
+    /**
+     * Computes the checkpoint plan 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 plan of this checkpoint.
+     */
+    private CheckpointPlan calculateWithAllTasksRunning() {
+        List<Execution> executionsToTrigger =
+                sourceTasks.stream()
+                        .map(ExecutionVertex::getCurrentExecutionAttempt)
+                        .collect(Collectors.toList());
+
+        Map<ExecutionAttemptID, ExecutionVertex> ackTasks = 
createTaskToAck(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableMap(ackTasks),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Computes the checkpoint plan 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 plan of this checkpoint.
+     */
+    private CheckpointPlan 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);
+                Arrays.stream(jobVertex.getTaskVertices())
+                        .forEach(task -> 
finishedTasks.add(task.getCurrentExecutionAttempt()));
+                continue;
+            }
+
+            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 CheckpointPlan(
+                Collections.unmodifiableList(tasksToTrigger),
+                Collections.unmodifiableMap(tasksToAck),
+                Collections.unmodifiableList(
+                        tasksToAck.size() == allTasks.size()
+                                ? allTasks
+                                : new ArrayList<>(tasksToAck.values())),
+                Collections.unmodifiableList(finishedTasks),
+                Collections.unmodifiableList(fullyFinishedJobVertex));
+    }
 
-        return executionsToTrigger;
+    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)

Review comment:
       Does this mean that for distribution pattern ALL_TO_ALL 
`hasRunningPrecedentTasks` returns false?
   
   Shouldn't it be `true` if there is at least one running upstream subtask 
(regardless of the distribution pattern)? So that we don't trigger a subtask if 
it has at least one active upstream?
   
   Edit:
   I see `someTasksMustBeTriggered` already covers this case.




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