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: [email protected]
