curcur commented on code in PR #22890: URL: https://github.com/apache/flink/pull/22890#discussion_r1327104203
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorFileMergingManager.java: ########## @@ -0,0 +1,146 @@ +/* + * 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.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; + +/** + * There is one {@link FileMergingSnapshotManager} for each job per task manager. This class holds + * all {@link FileMergingSnapshotManager} objects for a task executor (manager). + */ +public class TaskExecutorFileMergingManager { + /** Logger for this class. */ + private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorFileMergingManager.class); + + /** + * This map holds all FileMergingSnapshotManager for tasks running on this task + * manager(executor). + */ + @GuardedBy("lock") + private final Map<JobID, FileMergingSnapshotManager> fileMergingSnapshotManagerByJobId; + + @GuardedBy("lock") + private boolean closed; + + private final Object lock = new Object(); + + /** Shutdown hook for this manager. */ + private final Thread shutdownHook; + + public TaskExecutorFileMergingManager() { + this.fileMergingSnapshotManagerByJobId = new HashMap<>(); + this.closed = false; + this.shutdownHook = + ShutdownHookUtil.addShutdownHook(this::shutdown, getClass().getSimpleName(), LOG); + } + + /** + * Initialize file merging snapshot manager for each job according configurations when {@link + * org.apache.flink.runtime.taskexecutor.TaskExecutor#submitTask}. + */ + public @Nullable FileMergingSnapshotManager fileMergingSnapshotManagerForJob( + @Nonnull JobID jobId, + Configuration clusterConfiguration, + Configuration jobConfiguration) { Review Comment: this two parameters are not used add together with "FLINK-32440"? ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageAccess.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.filesystem; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager.SubtaskKey; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** An implementation of file merging checkpoint storage to file systems. */ +public class FsMergingCheckpointStorageAccess extends FsCheckpointStorageAccess { + + /** FileMergingSnapshotManager manages files and meta information for checkpoints. */ + private final FileMergingSnapshotManager fileMergingSnapshotManager; + + /** The identity of subtask. */ + private final FileMergingSnapshotManager.SubtaskKey subtaskKey; + + public FsMergingCheckpointStorageAccess( + FileSystem fs, + Path checkpointBaseDirectory, + @Nullable Path defaultSavepointDirectory, + JobID jobId, + int fileSizeThreshold, + int writeBufferSize, + FileMergingSnapshotManager fileMergingSnapshotManager, + Environment environment) + throws IOException { + super( + fs, + checkpointBaseDirectory, + defaultSavepointDirectory, + jobId, + fileSizeThreshold, + writeBufferSize); + this.fileMergingSnapshotManager = fileMergingSnapshotManager; + this.subtaskKey = + new SubtaskKey( + OperatorID.fromJobVertexID(environment.getJobVertexId()), + environment.getTaskInfo()); + } + + public FileMergingSnapshotManager getSegmentSnapshotManager() { + return fileMergingSnapshotManager; Review Comment: why the function name and member function is different? ########## 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 each job per task manager. Review Comment: If a job has multiple tasks on the same TM (not sub-tasks), do they share the same FileMergingSnapshotManager? I mean something like "(Source 0/1)"" (MAP 0/1)", but source and map belong to the same job -- 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]
