You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/24 11:54:08 UTC

[GitHub] [flink] gaoyunhaii opened a new pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

gaoyunhaii opened a new pull request #14740:
URL: https://github.com/apache/flink/pull/14740


   ## What is the purpose of the change
   
   Modifies the logic of computation of checkpoint brief to support the case that have finished tasks. 
   
   **Basic Algorithm**
   
   The algorithm would search the graph from the source vertices to find those tasks that
   1. Is still running.
   2. Does not have running precedent tasks.
   
   To reduce the overhead, the search is done in JobVertex level instead of task level. It keeps reduce the possible tasks that need to trigger for each JobVertex via the observation that
   1. If JobVertex `A` has running tasks and `A->B` via a `ALL_TO_ALL` edges, then all the tasks of B must be still running and do not need to be triggered. 
   2. If JobVertex `A` has running tasks and `A->B` via a `POINTWISE` edges, then only the descendant tasks of finished tasks of `A` might need to be triggered. 
   
   It could be seen the time complexity for the whole algorithm would be linear to the number of tasks instead of execution edges.
   
   **Considering the unordered finished report**
   
   Another note about the algorithm is that since each tasks would report FINISHED to JM without coordination, thus for graph like `A -> B -> C`, it is possible that JobMaster first received the `FINISHED` report of `A` and `C` before `B`.  This might cause problem for the basic algorithm. Thus we would first iterates the graph reversely to find the accurate set of running tasks according to the observation that:
   1. If one task is finished, all its precedent tasks are finished.
   
   The iteration is also done in JobVertex level, like the basic algorithm. Thus as a whole, the computation need to iterates the graph for two times.
   
   **Flag to disable checkpoints after tasks**
   
   Checkpoints after tasks finished need to be enabled as a whole after we have finished all the work for JM, TM and failover, otherwise it may cause failed checkpoints or in-consistent checkpoint. To avoid this issue, we would introduce a temporary flag in `CheckpointCoordinator` to disable checkpoint after tasks finished in formal process for now and would remove the flags after the whole development is done.
   
   ## Brief change log
   
   - e6bb7ea2b9ecde7e33fd4412dac891a272f790a6 modifies the formal process
   - 2e4dd30cea5745047c2fda23ddc3bda442ecb30a fixed the existing tests
   - fc9eebe10a60640303df43899888d783af9293a0 added new tests for the changed logic.
   
   ## Verifying this change
   
   - Added UT to verify the computation of checkpoint brief and checkpoint status tracker with finished tasks.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): **no**
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: **no**
     - The serializers: **no**
     - The runtime per-record code paths (performance sensitive): **no**
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: **no**
     - The S3 file system connector: **no**
   
   ## Documentation
   
     - Does this pull request introduce a new feature? **no**
     - If yes, how is the feature documented? **not applicable**
   


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-781124923


   Hi Roman @rkhachatryan , I updated the PR according to the offline discussion:
   
   1. Introduce the interface for `CheckpointPlanCalculator` for future extension.
   2. Do not consider the unreported finished tasks.
   3. Change java stream to iteration.
   4. Change the `tasksToWaitFor` from map to list.
   
   Could you have another look? Very thanks!


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570867096



##########
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:
       This is due to that for example, when we are judge whether a task is in fact finished:
   1. If it is connected with downstream tasks with ALL_TO_ALL edges, then all the tasks of this vertex would need to send EndOfPartition to all its descendants, thus if _any_ of the descendants is finished, we know all the tasks of this vertex has sent EndOfPartition (otherwise the downstream tasks cannot be finished).
   2. But it would not hold for pointwise edges since each task of this vertex only connects to a part of the downstream tasks. Only when _all_ the tasks of the downstream tasks are finished, we could judge that all the tasks of this vertex has finished. 
   
   These two would serve as a fast path so that we do not need to judge each task separately if possible. But if it cannot decide, we would then head to judge each task separately~

##########
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:
       No, I'm previously worrying about remove `log.info`, do you think it would has issue if some users rely on this message but it is changed~? But indeed I do not have a strong argument for it. 




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15a1f2beef1ddf95687fb4a53cf8a6f906559836 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r579054905



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());

Review comment:
       Thanks for updating, I see `hasRunningPrecedentTasks` now doesn't access `volatile` field.
   However, 2nd access is still there (`calculateAfterTasksFinished`).




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 3127cc7d9cb2eed93319e62b13e6b6b13175b82b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570882885



##########
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 see. But it seems unlikely to me that someone is relying on this message (because it's `INFO` and there is nothing user can do).
   WDYT?




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



[GitHub] [flink] guoweiM closed pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
guoweiM closed pull request #14740:
URL: https://github.com/apache/flink/pull/14740


   


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   * 06f660b73c75590ec1867a578504486531ece624 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750






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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570818510



##########
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:
       Highlighting finished subtasks can be helpful, but I agree it can be done later. 




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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r578907964



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());

Review comment:
       Very thanks for the comment! It should indeed more effective since in `hasRunningPrecedentTasks` we may need to access the `isFinished` for multiple times. According to the previous tests, I recorded the running status in `BitSet`, it should be able to compress the memory used and also avoid the time used for dynamic memory allocation~




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62efb51ddec76f3511f84038b3a6e7c5de4ee49c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782) 
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570878198



##########
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 understand, thanks!




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb37c3a640e8d8cd4cd969f413748387a55cccc3 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779",
       "triggerID" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13807",
       "triggerID" : "786327409",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 638f2d604054893ccce676ba35c5343d3a2f17b4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13807) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r578872539



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());
+
+                    if (someTasksMustBeTriggered) {
+                        boolean hasRunningPrecedentTasks =
+                                hasRunningPrecedentTasks(vertex, prevJobEdges);
+
+                        if (!hasRunningPrecedentTasks) {
+                            tasksToTrigger.add(vertex.getCurrentExecutionAttempt());
+                        }
+                    }
+                } else {
+                    finishedTasks.add(vertex.getCurrentExecutionAttempt());
+                }
+            }
+        }
+
+        List<ExecutionVertex> tasksToCommitTo;
+        if (tasksToWaitFor.size() == allTasks.size()) {
+            tasksToCommitTo = allTasks;
+        } else {
+            tasksToCommitTo = new ArrayList<>(tasksToWaitFor.size());
+            for (Execution execution : tasksToWaitFor) {
+                tasksToCommitTo.add(execution.getVertex());

Review comment:
       Yes, it is indeed not reachable... I moved it to the loop.




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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-770215757


   Very thanks @guoweiM for the comments and I have update the algorithm accordingly


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-771480929


   Hi @aljoscha I like the two patches for they make the calculation process much more clear. I have updated the PR according to the comments. 


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15a1f2beef1ddf95687fb4a53cf8a6f906559836 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478) 
   * 2ab49d334140ab47d7eb144964fac540e6f98444 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 743d1592db1b1f62ef6e2b208517438e2fab3a66 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849) 
   * 0a5a79498ab93134eccbe025489ede9aae233392 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570712494



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




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 1b4d1fc172e44377cbde71a71f34ea7f17b722ce Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685) 
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 743d1592db1b1f62ef6e2b208517438e2fab3a66 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849) 
   * 0a5a79498ab93134eccbe025489ede9aae233392 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   * 06f660b73c75590ec1867a578504486531ece624 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830) 
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 743d1592db1b1f62ef6e2b208517438e2fab3a66 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 0a5a79498ab93134eccbe025489ede9aae233392 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750






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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570714613



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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 1d0b7e0393c2f52a49519d1bb03fad8ad349f899 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082) 
   * eb37c3a640e8d8cd4cd969f413748387a55cccc3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 1d0b7e0393c2f52a49519d1bb03fad8ad349f899 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570818510



##########
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:
       Highlighting finished subtasks can be helpful, but I agree it can be done later. 

##########
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:
       Are you concerned about logging this `CheckpointException` with `NOT_ALL_REQUIRED_TASKS_RUNNING` reason?
   
   I actually meant to add message to the Exception and remove logging statement:
   ```
   // LOG.info <- no logging here
   throw new CheckpointException(
           message,
           CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
   ```
   
   Not
   ```
   LOG.info(message, checkpointException);
   throw checkpointException;
   ```

##########
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'd also rename the variable.

##########
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:
       Yes, you are right.
   I'd still extract these checks into a method or two methods:
   ```
   private boolean haveConnection(
           DistributionPattern distribution, JobVertexTaskSet tasks, boolean active) {
       switch (distribution) {
           case POINTWISE:
               return active ? tasks.containsAllTasks() : tasks.containsNoTasks();
           case ALL_TO_ALL:
               return active ? !tasks.containsNoTasks() : !tasks.containsAllTasks();
           default:
               throw new IllegalArgumentException(distribution.name());
       }
   }
   ```
   
   or
   
   ```
   private boolean hasActiveUpstream(DistributionPattern distribution, JobVertexTaskSet up) {
       return (distribution == ALL_TO_ALL && !up.containsNoTasks())
               || (distribution == POINTWISE && up.containsAllTasks());
   }
   
   private boolean hasInactiveDownstream(DistributionPattern distribution, JobVertexTaskSet down) {
       return (distribution == ALL_TO_ALL && !down.containsAllTasks())
               || (distribution == POINTWISE && down.containsNoTasks());
   }
   ```
   WDYT?
   

##########
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:
       Do we really need to differentiate between ALL_TO_ALL and POINTWISE? Why can't we use the same branch for POINTWISE as we use for  ALL_TO_ALL?




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 0a5a79498ab93134eccbe025489ede9aae233392 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975) 
   * 1d0b7e0393c2f52a49519d1bb03fad8ad349f899 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   * 06f660b73c75590ec1867a578504486531ece624 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-786327409


   @flinkbot run azure


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r568375584



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefComputerContext.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.concurrent.ScheduledExecutor;
+
+/** Provides the context for {@link CheckpointBriefComputer} to compute the brief of checkpoints. */
+public interface CheckpointBriefComputerContext {
+
+    /**
+     * Acquires the main thread executor for this job.
+     *
+     * @return The main thread executor.
+     */
+    ScheduledExecutor getMainExecutor();

Review comment:
       I'm a bit concerns here since that`CheckpointCoordinator` uses `executor` to refer to the IO executor in JM side, would it be better if we distinguish them explicitly here ?

##########
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:
       It is currently used in `calculateCheckpointBrief()` to disable checkpoints after tasks finished. Previously I'm thinking to enable this flag in tests, but now I realized it would makes more sense to keep the tests and the normal process consistent. I think it reveals some tests are missed for our scenarios and I'll complement the tests. 




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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
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?

##########
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)
+                .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 upstreamRunningTasks =
+                    runningTasksByVertex.get(jobEdge.getSource().getProducer().getID());
+
+            if (hasActiveUpstream(distributionPattern, upstreamRunningTasks)) {
+                return false;
+            }
+        }
+        return true;
     }
 
     /**
-     * Check if all tasks that need to acknowledge the checkpoint are running. If not, abort the
-     * checkpoint
+     * Every task must have active upstream tasks if
      *
-     * @return the execution vertices which should give an ack response
-     * @throws CheckpointException the exception fails checking
+     * <ol>
+     *   <li>ALL_TO_ALL connection and some predecessors are still running.
+     *   <li>POINTWISE connection and all predecessors are still running.
+     * </ol>
+     *
+     * @param distribution The distribution pattern between the upstream vertex and the current
+     *     vertex.
+     * @param upstream The set of running tasks of the upstream vertex.
+     * @return Whether every task of the current vertex is connected to some active predecessors.
      */
-    private Map<ExecutionAttemptID, ExecutionVertex> getAckTasks() throws CheckpointException {
-        Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new HashMap<>(tasksToWait.size());
+    private boolean hasActiveUpstream(DistributionPattern distribution, JobVertexTaskSet upstream) {
+        return (distribution == DistributionPattern.ALL_TO_ALL && !upstream.containsNoTasks())
+                || (distribution == DistributionPattern.POINTWISE && upstream.containsAllTasks());
+    }
 
-        for (ExecutionVertex ev : tasksToWait) {
-            Execution ee = ev.getCurrentExecutionAttempt();
-            if (ee != null) {
-                ackTasks.put(ee.getAttemptId(), ev);
+    /**
+     * Compute the accurate running tasks for each job vertex. Currently if multiple tasks all
+     * finished in short period, the order of their reports of FINISHED is nondeterministic, and
+     * some tasks may report FINISHED before all its precedent tasks have.
+     *
+     * <p>To overcome this issue we would iterates the graph first to acquire the accurate running
+     * tasks. We would iterate the job graph in reverse topological order, and for each job vertex,
+     * we would remove those precedent tasks that connected to finished tasks of this job vertex
+     * from possibly running tasks.
+     *
+     * @return An accurate set of running tasks for each job vertex.
+     */
+    @VisibleForTesting
+    Map<JobVertexID, JobVertexTaskSet> calculateRunningTasks() {
+        Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex = new HashMap<>();
+
+        ListIterator<ExecutionJobVertex> jobVertexIterator =
+                jobVerticesInTopologyOrder.listIterator(jobVerticesInTopologyOrder.size());
+
+        while (jobVertexIterator.hasPrevious()) {
+            ExecutionJobVertex jobVertex = jobVertexIterator.previous();
+
+            List<JobEdge> outputJobEdges = getOutputJobEdges(jobVertex);
+
+            // we're lucky if this is true
+            if (isFinishedAccordingToDescendants(runningTasksByVertex, outputJobEdges)) {
+                runningTasksByVertex.put(
+                        jobVertex.getJobVertexId(), JobVertexTaskSet.noTasks(jobVertex));
+                continue;
+            }
+
+            // not lucky, need to determine which of our tasks can still be running
+            Set<ExecutionVertexID> runningTasks =
+                    getRunningTasks(runningTasksByVertex, jobVertex, outputJobEdges);
+
+            runningTasksByVertex.put(
+                    jobVertex.getJobVertexId(),
+                    JobVertexTaskSet.someTasks(jobVertex, runningTasks));
+        }
+
+        return runningTasksByVertex;
+    }
+
+    /**
+     * Determines the {@link ExecutionVertexID ExecutionVertexIDs} of those subtasks that are still
+     * running.
+     */
+    private Set<ExecutionVertexID> getRunningTasks(
+            Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex,
+            ExecutionJobVertex jobVertex,
+            List<JobEdge> outputJobEdges) {
+        return Arrays.stream(jobVertex.getTaskVertices())
+                .filter(
+                        task -> {
+                            // the task is done if the execution graph thinks it is done
+                            if (task.getCurrentExecutionAttempt().isFinished()) {
+                                return false;
+                            }
+
+                            for (JobEdge edge : outputJobEdges) {
+                                if (edge.getDistributionPattern()
+                                        == DistributionPattern.POINTWISE) {

Review comment:
       Why do we check this only for `POINTWISE` connections? For `ALL_TO_ALL` downstream subtask to stop, all its upstreams must send `EndOfPartition`, no?
   
   Or is it an optimization to avoid O(n^2)?




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438) 
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * ea70d7913106bb528aaf3e775679619a9f241be9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r573992537



##########
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)
+                .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 upstreamRunningTasks =
+                    runningTasksByVertex.get(jobEdge.getSource().getProducer().getID());
+
+            if (hasActiveUpstream(distributionPattern, upstreamRunningTasks)) {
+                return false;
+            }
+        }
+        return true;
     }
 
     /**
-     * Check if all tasks that need to acknowledge the checkpoint are running. If not, abort the
-     * checkpoint
+     * Every task must have active upstream tasks if
      *
-     * @return the execution vertices which should give an ack response
-     * @throws CheckpointException the exception fails checking
+     * <ol>
+     *   <li>ALL_TO_ALL connection and some predecessors are still running.
+     *   <li>POINTWISE connection and all predecessors are still running.
+     * </ol>
+     *
+     * @param distribution The distribution pattern between the upstream vertex and the current
+     *     vertex.
+     * @param upstream The set of running tasks of the upstream vertex.
+     * @return Whether every task of the current vertex is connected to some active predecessors.
      */
-    private Map<ExecutionAttemptID, ExecutionVertex> getAckTasks() throws CheckpointException {
-        Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new HashMap<>(tasksToWait.size());
+    private boolean hasActiveUpstream(DistributionPattern distribution, JobVertexTaskSet upstream) {
+        return (distribution == DistributionPattern.ALL_TO_ALL && !upstream.containsNoTasks())
+                || (distribution == DistributionPattern.POINTWISE && upstream.containsAllTasks());
+    }
 
-        for (ExecutionVertex ev : tasksToWait) {
-            Execution ee = ev.getCurrentExecutionAttempt();
-            if (ee != null) {
-                ackTasks.put(ee.getAttemptId(), ev);
+    /**
+     * Compute the accurate running tasks for each job vertex. Currently if multiple tasks all
+     * finished in short period, the order of their reports of FINISHED is nondeterministic, and
+     * some tasks may report FINISHED before all its precedent tasks have.
+     *
+     * <p>To overcome this issue we would iterates the graph first to acquire the accurate running
+     * tasks. We would iterate the job graph in reverse topological order, and for each job vertex,
+     * we would remove those precedent tasks that connected to finished tasks of this job vertex
+     * from possibly running tasks.
+     *
+     * @return An accurate set of running tasks for each job vertex.
+     */
+    @VisibleForTesting
+    Map<JobVertexID, JobVertexTaskSet> calculateRunningTasks() {
+        Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex = new HashMap<>();
+
+        ListIterator<ExecutionJobVertex> jobVertexIterator =
+                jobVerticesInTopologyOrder.listIterator(jobVerticesInTopologyOrder.size());
+
+        while (jobVertexIterator.hasPrevious()) {
+            ExecutionJobVertex jobVertex = jobVertexIterator.previous();
+
+            List<JobEdge> outputJobEdges = getOutputJobEdges(jobVertex);
+
+            // we're lucky if this is true
+            if (isFinishedAccordingToDescendants(runningTasksByVertex, outputJobEdges)) {
+                runningTasksByVertex.put(
+                        jobVertex.getJobVertexId(), JobVertexTaskSet.noTasks(jobVertex));
+                continue;
+            }
+
+            // not lucky, need to determine which of our tasks can still be running
+            Set<ExecutionVertexID> runningTasks =
+                    getRunningTasks(runningTasksByVertex, jobVertex, outputJobEdges);
+
+            runningTasksByVertex.put(
+                    jobVertex.getJobVertexId(),
+                    JobVertexTaskSet.someTasks(jobVertex, runningTasks));
+        }
+
+        return runningTasksByVertex;
+    }
+
+    /**
+     * Determines the {@link ExecutionVertexID ExecutionVertexIDs} of those subtasks that are still
+     * running.
+     */
+    private Set<ExecutionVertexID> getRunningTasks(
+            Map<JobVertexID, JobVertexTaskSet> runningTasksByVertex,
+            ExecutionJobVertex jobVertex,
+            List<JobEdge> outputJobEdges) {
+        return Arrays.stream(jobVertex.getTaskVertices())
+                .filter(
+                        task -> {
+                            // the task is done if the execution graph thinks it is done
+                            if (task.getCurrentExecutionAttempt().isFinished()) {
+                                return false;
+                            }
+
+                            for (JobEdge edge : outputJobEdges) {
+                                if (edge.getDistributionPattern()
+                                        == DistributionPattern.POINTWISE) {

Review comment:
       Why do we check this only for `POINTWISE` connections? For `ALL_TO_ALL` downstream subtask to stop, all its upstreams must send `EndOfPartition`, no?
   
   Or is it an optimization to avoid O(n^2)?
   
   Edit:
   It's the same as above: covered by `isFinishedAccordingToDescendants` :) 




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



[GitHub] [flink] gaoyunhaii edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-771480929


   Hi @aljoscha Very thanks for the review and patches, I like the patches for they make the calculation process much more clear. I have updated the PR according to the comments. 


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570694404



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




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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570829287



##########
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:
       Are you concerned about logging this `CheckpointException` with `NOT_ALL_REQUIRED_TASKS_RUNNING` reason?
   
   I actually meant to add message to the Exception and remove logging statement:
   ```
   // LOG.info <- no logging here
   throw new CheckpointException(
           message,
           CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
   ```
   
   Not
   ```
   LOG.info(message, checkpointException);
   throw checkpointException;
   ```




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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r568375584



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefComputerContext.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.concurrent.ScheduledExecutor;
+
+/** Provides the context for {@link CheckpointBriefComputer} to compute the brief of checkpoints. */
+public interface CheckpointBriefComputerContext {
+
+    /**
+     * Acquires the main thread executor for this job.
+     *
+     * @return The main thread executor.
+     */
+    ScheduledExecutor getMainExecutor();

Review comment:
       I'm a bit concerns here since that`CheckpointCoordinator` uses `executor` to refer to the IO executor in JM side, would it be better if we distinguish them explicitly here ?




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750) 
   * 62efb51ddec76f3511f84038b3a6e7c5de4ee49c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782) 
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r578709529



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -651,39 +681,52 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) {
         }
     }
 
+    private CompletableFuture<CheckpointPlan> calculateCheckpointPlan() {
+        return checkpointPlanCalculator
+                .calculateCheckpointPlan()
+                // Disable checkpoints after tasks finished according to the flag.
+                .thenApplyAsync(
+                        plan -> {
+                            if (!allowCheckpointsAfterTasksFinished
+                                    && !plan.getFinishedTasks().isEmpty()) {
+                                throw new CompletionException(

Review comment:
       Should we do this check **before** computing the plan? 
   Otherwise, 
   1. the flag doesn't prevent existing deployments from potential performance degradation
   2. the plan is computed for no purpose
   
   I think it can also be put it planCalculator if it's easier.
   
   WDYT?




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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r574247927



##########
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:
       Yes, exactly, we want to use `someTasksMustBeTriggered` to cover this case so that we could avoid the `O(n^2)` 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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2ab49d334140ab47d7eb144964fac540e6f98444 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694) 
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 0a5a79498ab93134eccbe025489ede9aae233392 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975) 
   * 1d0b7e0393c2f52a49519d1bb03fad8ad349f899 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766338671


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit fc9eebe10a60640303df43899888d783af9293a0 (Sun Jan 24 12:17:03 UTC 2021)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750) 
   * 62efb51ddec76f3511f84038b3a6e7c5de4ee49c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2ab49d334140ab47d7eb144964fac540e6f98444 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779",
       "triggerID" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) 
   * 638f2d604054893ccce676ba35c5343d3a2f17b4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-773782948


   Hi Roman @rkhachatryan very thanks for the review! I have update the PR via https://github.com/apache/flink/pull/14740/commits/0a5a79498ab93134eccbe025489ede9aae233392 according to the comments~
   
   The current PR indeed did not include the case that the task finishes concurrently when JM tries to trigger it, [FLINK-21246](https://issues.apache.org/jira/browse/FLINK-21246) would solve this issue. I also think in this case the checkpoint would be declined with a reason that would not cause job failure. 


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r569605411



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -673,45 +694,62 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) {
         }
     }
 
+    private CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        return checkpointBriefCalculator
+                .calculateCheckpointBrief()
+                // Disable checkpoints after tasks finished according to the flag.
+                .thenApplyAsync(
+                        brief -> {
+                            if (brief.getFinishedTasks().size() > 0
+                                    && !allowCheckpointsAfterTasksFinished) {
+                                LOG.info(
+                                        "{} tasks of job {} has been finished, abort the checkpoint",
+                                        brief.getFinishedTasks().size(),
+                                        job);
+                                throw new CompletionException(
+                                        new CheckpointException(
+                                                CheckpointFailureReason
+                                                        .NOT_ALL_REQUIRED_TASKS_RUNNING));
+                            }
+
+                            return brief;
+                        },
+                        timer);
+    }
+
     /**
-     * Initialize the checkpoint trigger asynchronously. It will be executed in io thread due to it
-     * might be time-consuming.
+     * Initialize the checkpoint trigger asynchronously. It will expected to be executed in io
+     * thread due to it might be time-consuming.
      *
      * @param props checkpoint properties
      * @param externalSavepointLocation the external savepoint location, it might be null
-     * @return the future of initialized result, checkpoint id and checkpoint location
+     * @return the initialized result, checkpoint id and checkpoint location
      */
-    private CompletableFuture<CheckpointIdAndStorageLocation> initializeCheckpoint(
+    private CheckpointIdAndStorageLocation initializeCheckpoint(
             CheckpointProperties props, @Nullable String externalSavepointLocation) {
 
-        return CompletableFuture.supplyAsync(
-                () -> {
-                    try {
-                        // this must happen outside the coordinator-wide lock, because it
-                        // communicates
-                        // with external services (in HA mode) and may block for a while.
-                        long checkpointID = checkpointIdCounter.getAndIncrement();
-
-                        CheckpointStorageLocation checkpointStorageLocation =
-                                props.isSavepoint()
-                                        ? checkpointStorageView.initializeLocationForSavepoint(
-                                                checkpointID, externalSavepointLocation)
-                                        : checkpointStorageView.initializeLocationForCheckpoint(
-                                                checkpointID);
-
-                        return new CheckpointIdAndStorageLocation(
-                                checkpointID, checkpointStorageLocation);
-                    } catch (Throwable throwable) {
-                        throw new CompletionException(throwable);
-                    }
-                },
-                executor);
+        try {
+            // this must happen outside the coordinator-wide lock, because it
+            // communicates
+            // with external services (in HA mode) and may block for a while.
+            long checkpointID = checkpointIdCounter.getAndIncrement();
+
+            CheckpointStorageLocation checkpointStorageLocation =
+                    props.isSavepoint()
+                            ? checkpointStorageView.initializeLocationForSavepoint(
+                                    checkpointID, externalSavepointLocation)
+                            : checkpointStorageView.initializeLocationForCheckpoint(checkpointID);
+
+            return new CheckpointIdAndStorageLocation(checkpointID, checkpointStorageLocation);
+        } catch (Throwable throwable) {
+            throw new CompletionException(throwable);

Review comment:
       There is one more try/catch level above now. I think this one can be dropped.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -673,45 +694,62 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) {
         }
     }
 
+    private CompletableFuture<CheckpointBrief> calculateCheckpointBrief() {
+        return checkpointBriefCalculator
+                .calculateCheckpointBrief()
+                // Disable checkpoints after tasks finished according to the flag.
+                .thenApplyAsync(
+                        brief -> {
+                            if (brief.getFinishedTasks().size() > 0
+                                    && !allowCheckpointsAfterTasksFinished) {

Review comment:
       nit: `isEmpty()` and swap `&&`?

##########
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:
       Assert main JM thread (or at least comment)?

##########
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:
       It would be easier to follow if we continue the iteration in this case:
   ```
   for (ExecutionVertex ev: jobVertex.getTaskVertices()) {
       finishedTasks.add(ev.getCurrentExecutionAttempt());
   }
   continue;
   ```
   (or if-else instead of `continue`)
   

##########
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:
       Could you please clarify:
   - This is NOT a feature toggle and will be removed in the final PR?
   - In this PR, it will be disabled before merging into master?
   
   (if not, I think it should be `final` and have a corresponding (hidden for now) `ConfigOption`)

##########
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:
       nit: `getFinishedVertices`?

##########
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:
       We can avoid explicit creation and completion of a future with:
   ```
   return CompletableFuture.supplyAsync(
       () -> {
           ...
       },
       context.getMainExecutor());
   ```
   WDYT?

##########
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:
       This will report the tasks as it completed the checkpoint, right?
   Is it a desired behavior?

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

Review comment:
       WDYT about throwing an exception with this message directly from this method? The only usage does this, but loses the message.
   
   ditto `isAllExecutionAttemptsAreInitiated`

##########
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:
       It's hard to read this name to me.
   If it throws an exception instead of returning booleain (see below)
   then I'd rename it to `checkAllStarted`
   
   ditto `isAllExecutionAttemptsAreInitiated` -> `checkAllTasksInitiated`

##########
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();
+                                }

Review comment:
       nit:
   ```
   CheckpointBrief result =
           context.hasFinishedTasks()
                   ? calculateAfterTasksFinished()
                   : calculateWithAllTasksRunning();
   ```

##########
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:
       Could you extract a function
   ```
   private boolean hasConnection(DistributionPattern distribution, JobVertexTaskSet runningTasks) {
       switch (distribution) {
           case ALL_TO_ALL:
               return !runningTasks.containsAllTasks();
           case POINTWISE:
               return runningTasks.containsNoTasks();
           default:
               throw new IllegalArgumentException(distribution.name());
       }
   }
   ```
   and use it here and in `isFinishedAccordingToDescendants`?




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



[GitHub] [flink] gaoyunhaii edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-774912042


   Hi Roman @rkhachatryan, very thanks for the careful review! I updated the remaining issue with https://github.com/apache/flink/pull/14740/commits/1d0b7e0393c2f52a49519d1bb03fad8ad349f899.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb37c3a640e8d8cd4cd969f413748387a55cccc3 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099) 
   * c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438) 
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * ea70d7913106bb528aaf3e775679619a9f241be9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fc9eebe10a60640303df43899888d783af9293a0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422) 
   * 15a1f2beef1ddf95687fb4a53cf8a6f906559836 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-781798040


   Hi Roman @rkhachatryan very thanks for the review! I updated the PR according to the comments~


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570713863



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




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



[GitHub] [flink] gaoyunhaii edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-773782948


   Hi Roman @rkhachatryan very thanks for the review! I have update the PR via https://github.com/apache/flink/pull/14740/commits/0a5a79498ab93134eccbe025489ede9aae233392 according to the comments~
   
   The current PR indeed did not include the case that the task finishes concurrently when JM tries to trigger it, [FLINK-21246](https://issues.apache.org/jira/browse/FLINK-21246) would solve this issue. I also think in this case the checkpoint would be declined with a reason that would not cause job failure. 


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 3127cc7d9cb2eed93319e62b13e6b6b13175b82b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493) 
   * 1b4d1fc172e44377cbde71a71f34ea7f17b722ce Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * ea70d7913106bb528aaf3e775679619a9f241be9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 743d1592db1b1f62ef6e2b208517438e2fab3a66 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779",
       "triggerID" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 638f2d604054893ccce676ba35c5343d3a2f17b4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-773782948


   Hi Roman @rkhachatryan very thanks for the review! I have update the PR via https://github.com/apache/flink/pull/14740/commits/0a5a79498ab93134eccbe025489ede9aae233392 according to the comments~


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) 
   * 638f2d604054893ccce676ba35c5343d3a2f17b4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750) 
   * 62efb51ddec76f3511f84038b3a6e7c5de4ee49c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782) 
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-773782948


   Hi Roman @rkhachatryan very thanks for the review! I have update the PR via https://github.com/apache/flink/pull/14740/commits/0a5a79498ab93134eccbe025489ede9aae233392 according to the comments~


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * ea70d7913106bb528aaf3e775679619a9f241be9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484) 
   * 3127cc7d9cb2eed93319e62b13e6b6b13175b82b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   * 06f660b73c75590ec1867a578504486531ece624 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830) 
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 743d1592db1b1f62ef6e2b208517438e2fab3a66 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 1b4d1fc172e44377cbde71a71f34ea7f17b722ce Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r578709529



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -651,39 +681,52 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) {
         }
     }
 
+    private CompletableFuture<CheckpointPlan> calculateCheckpointPlan() {
+        return checkpointPlanCalculator
+                .calculateCheckpointPlan()
+                // Disable checkpoints after tasks finished according to the flag.
+                .thenApplyAsync(
+                        plan -> {
+                            if (!allowCheckpointsAfterTasksFinished
+                                    && !plan.getFinishedTasks().isEmpty()) {
+                                throw new CompletionException(

Review comment:
       Should we do this check before computing the plan? 
   Otherwise, 
   1. the flag doesn't prevent existing deployments from potential performance degradation
   2. the plan is computed for no purpose
   
   I think it can also be put it planCalculator if it's easier.
   
   WDYT?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());
+
+                    if (someTasksMustBeTriggered) {
+                        boolean hasRunningPrecedentTasks =
+                                hasRunningPrecedentTasks(vertex, prevJobEdges);
+
+                        if (!hasRunningPrecedentTasks) {
+                            tasksToTrigger.add(vertex.getCurrentExecutionAttempt());
+                        }
+                    }
+                } else {
+                    finishedTasks.add(vertex.getCurrentExecutionAttempt());
+                }
+            }
+        }
+
+        List<ExecutionVertex> tasksToCommitTo;
+        if (tasksToWaitFor.size() == allTasks.size()) {
+            tasksToCommitTo = allTasks;
+        } else {
+            tasksToCommitTo = new ArrayList<>(tasksToWaitFor.size());
+            for (Execution execution : tasksToWaitFor) {
+                tasksToCommitTo.add(execution.getVertex());

Review comment:
       I guess this is a memory optimization. 
   But shouldn't we should optimize for CPU time here? (memory usage should be limited to a couple of Mb).
   For that, I'd move `tasksToCommitTo.add` to the loop above.
   
   And it looks like the 1st branch isn't reachable?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();

Review comment:
       Could you add a comment that two passes are needed to quiclky check whether subtask can be triggered? 
   (I think we should also use this map in `hasRunningPrecedentTasks`, see below)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());

Review comment:
       I think this can be a performance regression compared to the previous version. `isFinised` reads a `volatile` field and this is done `O(N)` times in the worst case.
   The same issue is in `hasRunningPrecedentTasks`.
   
   Can we avoid it if we revert `numberOfRunningTasksByVertex` to a map of subtasks?
   
   For example, `countRunningTasks()` could collect finished and running subtasks per vertex. Returning somthing like `Map<JobVertexID, Tuple2<List<ExecutionVertex>, List<ExecutionVertex>>>` (or some class instead of tuple).
   
   WDYT?




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * f291c334a6571adc1141a6201e2e531d3b221efa Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794) 
   * 06f660b73c75590ec1867a578504486531ece624 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830) 
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570686307



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




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750) 
   * 62efb51ddec76f3511f84038b3a6e7c5de4ee49c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
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.  




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9394c62c445d5a17f802d0b6eee51bce00d9df9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750) 
   * 62efb51ddec76f3511f84038b3a6e7c5de4ee49c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782) 
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570711043



##########
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`~?




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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570704750



##########
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~?




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15a1f2beef1ddf95687fb4a53cf8a6f906559836 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478) 
   * 2ab49d334140ab47d7eb144964fac540e6f98444 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] guoweiM commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
guoweiM commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-769026789


   After some offline discussion with @gaoyunhaii we reach a consensus that we should make the algorithm(Compute “tasksToTrigger/tasksToWaitFor” ) easy to understand. In general, we agree change the algorithm from “remove impossible trigger tasks” to “compute directly the trigger tasks.”
   There are still some little open questions but Yun would fix the algorithm first then I would take another look.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750






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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-774912042


   Hi Roman @rkhachatryan, very thanks for the careful review, I updated the remaining issue with https://github.com/apache/flink/pull/14740/commits/1d0b7e0393c2f52a49519d1bb03fad8ad349f899.


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779",
       "triggerID" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13807",
       "triggerID" : "786327409",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "638f2d604054893ccce676ba35c5343d3a2f17b4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779",
       "triggerID" : "786327409",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 638f2d604054893ccce676ba35c5343d3a2f17b4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13807) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13779) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438) 
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-785731866


   @flinkbot run azure


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735",
       "triggerID" : "785731866",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13754) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13735) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r570829925



##########
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'd also rename the 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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 1b4d1fc172e44377cbde71a71f34ea7f17b722ce Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13685) 
   * d2b929d9a6f8f9ce142d94ef8be40d8e70e289a1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] aljoscha commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r567923261



##########
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:
       This flag seems to not even be used anywhere, so far?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
##########
@@ -183,26 +167,48 @@ 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,
+            List<ExecutionVertex> finishedTasks,
+            long checkpointId,
+            long triggerTimestamp,
+            CheckpointProperties props) {
 
         ConcurrentHashMap<JobVertexID, TaskStateStats> taskStateStats =
-                createEmptyTaskStateStatsMap();
+                createEmptyTaskStateStatsMap(tasksToAck, finishedTasks);
 
         PendingCheckpointStats pending =
                 new PendingCheckpointStats(
                         checkpointId,
                         triggerTimestamp,
                         props,
-                        totalSubtaskCount,
+                        tasksToAck.size() + finishedTasks.size(),
                         taskStateStats,
                         new PendingCheckpointStatsCallback());
 
+        long now = System.currentTimeMillis();

Review comment:
       I think it would be better to put this into a separate private method, something like `reportFinishedTasks(pending, finishedTasks)`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointBriefComputerContext.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.concurrent.ScheduledExecutor;
+
+/** Provides the context for {@link CheckpointBriefComputer} to compute the brief of checkpoints. */
+public interface CheckpointBriefComputerContext {
+
+    /**
+     * Acquires the main thread executor for this job.
+     *
+     * @return The main thread executor.
+     */
+    ScheduledExecutor getMainExecutor();

Review comment:
       This should be just `getExecutor()`, because we don't care what type it is. Or `getJobMasterMainThreadExecutor()` to keep it consistent with `ExecutionGraph`.
   
   I tend to prefer the first because it's shorter. 😅




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



[GitHub] [flink] gaoyunhaii commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-771480929


   Hi @aljoscha I like the two patches for they make the calculation process much more clear. I have updated the PR according to the comments. 


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



[GitHub] [flink] rkhachatryan commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r579616439



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());

Review comment:
       No worries, thanks for fixing this.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fc9eebe10a60640303df43899888d783af9293a0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r568377165



##########
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:
       It is currently used in `calculateCheckpointBrief()` to disable checkpoints after tasks finished. Previously I'm thinking to enable this flag in tests, but now I realized it would makes more sense to keep the tests and the normal process consistent. I think it reveals some tests are missed for our scenarios and I'll complement the tests. 




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



[GitHub] [flink] flinkbot commented on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fc9eebe10a60640303df43899888d783af9293a0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb37c3a640e8d8cd4cd969f413748387a55cccc3 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099) 
   * c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 1d0b7e0393c2f52a49519d1bb03fad8ad349f899 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082) 
   * eb37c3a640e8d8cd4cd969f413748387a55cccc3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * 06f660b73c75590ec1867a578504486531ece624 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830) 
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * 743d1592db1b1f62ef6e2b208517438e2fab3a66 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r578879271



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -651,39 +681,52 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) {
         }
     }
 
+    private CompletableFuture<CheckpointPlan> calculateCheckpointPlan() {
+        return checkpointPlanCalculator
+                .calculateCheckpointPlan()
+                // Disable checkpoints after tasks finished according to the flag.
+                .thenApplyAsync(
+                        plan -> {
+                            if (!allowCheckpointsAfterTasksFinished
+                                    && !plan.getFinishedTasks().isEmpty()) {
+                                throw new CompletionException(

Review comment:
       I also think move the check before computing would be more reasonable. I have modified accordingly.




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * ea70d7913106bb528aaf3e775679619a9f241be9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484) 
   * 3127cc7d9cb2eed93319e62b13e6b6b13175b82b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r578879271



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -651,39 +681,52 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) {
         }
     }
 
+    private CompletableFuture<CheckpointPlan> calculateCheckpointPlan() {
+        return checkpointPlanCalculator
+                .calculateCheckpointPlan()
+                // Disable checkpoints after tasks finished according to the flag.
+                .thenApplyAsync(
+                        plan -> {
+                            if (!allowCheckpointsAfterTasksFinished
+                                    && !plan.getFinishedTasks().isEmpty()) {
+                                throw new CompletionException(

Review comment:
       I also think move the check before computing would be more reasonable. I have modified accordingly. For the long run, we would remove the judgement after we enabled the whole functionality~




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



[GitHub] [flink] gaoyunhaii commented on a change in pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #14740:
URL: https://github.com/apache/flink/pull/14740#discussion_r579066215



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculator.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.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.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation for {@link CheckpointPlanCalculator}. If all tasks are running, it
+ * directly marks all the sources as tasks to trigger, otherwise it would try to find the running
+ * tasks without running processors as tasks to trigger.
+ */
+public class DefaultCheckpointPlanCalculator implements CheckpointPlanCalculator {
+
+    private final JobID jobId;
+
+    private final CheckpointPlanCalculatorContext context;
+
+    private final List<ExecutionJobVertex> jobVerticesInTopologyOrder = new ArrayList<>();
+
+    private final List<ExecutionVertex> allTasks = new ArrayList<>();
+
+    private final List<ExecutionVertex> sourceTasks = new ArrayList<>();
+
+    public DefaultCheckpointPlanCalculator(
+            JobID jobId,
+            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()));
+
+                    if (jobVertex.getJobVertex().isInputVertex()) {
+                        sourceTasks.addAll(Arrays.asList(jobVertex.getTaskVertices()));
+                    }
+                });
+    }
+
+    @Override
+    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());
+    }
+
+    /**
+     * Checks if all tasks are attached with the current Execution already. This method should be
+     * called from JobMaster main thread executor.
+     *
+     * @throws CheckpointException if some tasks do not have attached Execution.
+     */
+    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);
+            }
+        }
+    }
+
+    /**
+     * 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());
+
+        List<Execution> tasksToWaitFor = createTaskToWaitFor(allTasks);
+
+        return new CheckpointPlan(
+                Collections.unmodifiableList(executionsToTrigger),
+                Collections.unmodifiableList(tasksToWaitFor),
+                Collections.unmodifiableList(allTasks),
+                Collections.emptyList(),
+                Collections.emptyList());
+    }
+
+    /**
+     * Calculates the checkpoint plan after some tasks have finished. We 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, Integer> numberOfRunningTasksByVertex = countRunningTasks();
+
+        List<Execution> tasksToTrigger = new ArrayList<>();
+
+        List<Execution> tasksToWaitFor = new ArrayList<>();
+        List<Execution> finishedTasks = new ArrayList<>();
+        List<ExecutionJobVertex> fullyFinishedJobVertex = new ArrayList<>();
+
+        for (ExecutionJobVertex jobVertex : jobVerticesInTopologyOrder) {
+            int numberOfRunningTasks = numberOfRunningTasksByVertex.get(jobVertex.getJobVertexId());
+
+            if (numberOfRunningTasks == 0) {
+                fullyFinishedJobVertex.add(jobVertex);
+
+                for (ExecutionVertex task : jobVertex.getTaskVertices()) {
+                    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(numberOfRunningTasksByVertex, prevJobEdges);
+
+            for (ExecutionVertex vertex : jobVertex.getTaskVertices()) {
+                if (!vertex.getCurrentExecutionAttempt().isFinished()) {
+                    tasksToWaitFor.add(vertex.getCurrentExecutionAttempt());

Review comment:
       Very sorry for missing change this part... I fixed this issue: https://github.com/apache/flink/pull/14740/commits/3127cc7d9cb2eed93319e62b13e6b6b13175b82b




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



[GitHub] [flink] flinkbot edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-766340750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fc9eebe10a60640303df43899888d783af9293a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12422",
       "triggerID" : "fc9eebe10a60640303df43899888d783af9293a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12478",
       "triggerID" : "15a1f2beef1ddf95687fb4a53cf8a6f906559836",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12694",
       "triggerID" : "2ab49d334140ab47d7eb144964fac540e6f98444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12750",
       "triggerID" : "e9394c62c445d5a17f802d0b6eee51bce00d9df9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12782",
       "triggerID" : "62efb51ddec76f3511f84038b3a6e7c5de4ee49c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7e6b28b249f85cf52740d5201a769e0982a60aa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12794",
       "triggerID" : "f291c334a6571adc1141a6201e2e531d3b221efa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "06f660b73c75590ec1867a578504486531ece624",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12830",
       "triggerID" : "06f660b73c75590ec1867a578504486531ece624",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bebd298009b12a9d5ac6518902f5534f8e00ff32",
       "triggerType" : "PUSH"
     }, {
       "hash" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12849",
       "triggerID" : "743d1592db1b1f62ef6e2b208517438e2fab3a66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12975",
       "triggerID" : "0a5a79498ab93134eccbe025489ede9aae233392",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13082",
       "triggerID" : "1d0b7e0393c2f52a49519d1bb03fad8ad349f899",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13099",
       "triggerID" : "eb37c3a640e8d8cd4cd969f413748387a55cccc3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13438",
       "triggerID" : "c69c79df69c5fa912b6d16e3cfa40edc3b9b82f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb6c10b0d339bfc92a540314e7c58cbf11a70dd9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13484",
       "triggerID" : "ea70d7913106bb528aaf3e775679619a9f241be9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493",
       "triggerID" : "3127cc7d9cb2eed93319e62b13e6b6b13175b82b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1b4d1fc172e44377cbde71a71f34ea7f17b722ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7e6b28b249f85cf52740d5201a769e0982a60aa UNKNOWN
   * bebd298009b12a9d5ac6518902f5534f8e00ff32 UNKNOWN
   * eb6c10b0d339bfc92a540314e7c58cbf11a70dd9 UNKNOWN
   * 3127cc7d9cb2eed93319e62b13e6b6b13175b82b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13493) 
   * 1b4d1fc172e44377cbde71a71f34ea7f17b722ce UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] gaoyunhaii edited a comment on pull request #14740: [FLINK-21067][runtime][checkpoint] Modify the logic of computing which tasks to trigger/ack/commit to support finished tasks

Posted by GitBox <gi...@apache.org>.
gaoyunhaii edited a comment on pull request #14740:
URL: https://github.com/apache/flink/pull/14740#issuecomment-771480929


   Hi @aljoscha Very thanks for the review and patches, I like the patches for they make the calculation process much more clear. I have updated the PR according to the comments. 


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