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