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