Zakelly commented on code in PR #22890: URL: https://github.com/apache/flink/pull/22890#discussion_r1246270263
########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManager.java: ########## @@ -31,7 +32,7 @@ * FileMergingSnapshotManager provides an interface to manage files and meta information for * checkpoint files with merging checkpoint files enabled. It manages the files for ONE single task * in TM, including all subtasks of this single task that running in this TM. There is one - * FileMergingSnapshotManager for each task per task manager. + * FileMergingSnapshotManager for task job per task manager. Review Comment: Do you mean 'for each job per task manager' here? ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorFileMergingManager.java: ########## @@ -0,0 +1,137 @@ +/* + * 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.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManagerBuilder; +import org.apache.flink.util.ShutdownHookUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; + +/** + * This class holds the all {@link FileMergingSnapshotManager} objects for a task executor + * (manager). + */ Review Comment: Maybe more explanation about the relationship between this class and `FileMergingSnapshotManager`. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java: ########## @@ -456,6 +456,12 @@ protected StreamTask( CheckpointStorageAccess checkpointStorageAccess = checkpointStorage.createCheckpointStorage(getEnvironment().getJobID()); + environment + .getTaskStateManager() + .initCheckpointFileSystem( + checkpointStorageAccess, + environment.getJobVertexId(), + environment.getTaskInfo()); Review Comment: A little bit strange here. I think it is better to provide a function to apply file-merging configuration and get a new `CheckpointStorageAccess` from the `environment.getTaskStateManager().getFileMergingSnapshotManager(xxx)` and `checkpointStorageAccess` here. The new `CheckpointStorageAccess` will access the checkpoint storage in file-merging way, leveraging the `FileMergingSnapshotManager`. However in this PR, the new `CheckpointStorageAccess` and corresponding `CheckpointStorageLocation` could just wrap existing classes and instance of `FileMergingSnapshotManager`, and only do proxy. 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
