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