Zakelly commented on code in PR #22590: URL: https://github.com/apache/flink/pull/22590#discussion_r1223779099
########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java: ########## @@ -0,0 +1,282 @@ +/* + * 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.filemerging; + +import org.apache.flink.core.fs.EntropyInjector; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.OutputStreamAndPath; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.filemerging.LogicalFile.LogicalFileId; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; + +/** Base implementation of {@link FileMergingSnapshotManager}. */ +public abstract class FileMergingSnapshotManagerBase implements FileMergingSnapshotManager { + + private static final Logger LOG = LoggerFactory.getLogger(FileMergingSnapshotManager.class); + + private final String id; + + protected final Executor ioExecutor; + + // file system and directories + protected FileSystem fs; + protected Path checkpointDir; + protected Path sharedStateDir; + protected Path taskOwnedStateDir; + + protected int writeBufferSize; + private boolean fileSystemInitiated = false; + + protected boolean syncAfterClosingLogicalFile; + + protected PhysicalFile.PhysicalFileDeleter physicalFileDeleter = this::deletePhysicalFile; + + private final Map<SubtaskKey, Path> managedSharedStateDir = new ConcurrentHashMap<>(); + + protected Path managedExclusiveStateDir; + + public FileMergingSnapshotManagerBase(String id, Executor ioExecutor) { + this.id = id; + this.ioExecutor = ioExecutor; + } + + @Override + public void initFileSystem( + FileSystem fileSystem, + Path checkpointBaseDir, + Path sharedStateDir, + Path taskOwnedStateDir) { + if (fileSystemInitiated) { + Preconditions.checkArgument( + checkpointBaseDir.equals(this.checkpointDir), + "The checkpoint base dir is not deterministic across subtasks."); + Preconditions.checkArgument( + sharedStateDir.equals(this.sharedStateDir), + "The shared checkpoint dir is not deterministic across subtasks."); + Preconditions.checkArgument( + taskOwnedStateDir.equals(this.taskOwnedStateDir), + "The task-owned checkpoint dir is not deterministic across subtasks."); + return; + } + this.fs = fileSystem; + this.checkpointDir = Preconditions.checkNotNull(checkpointBaseDir); + this.sharedStateDir = Preconditions.checkNotNull(sharedStateDir); + this.taskOwnedStateDir = Preconditions.checkNotNull(taskOwnedStateDir); + this.fileSystemInitiated = true; + this.syncAfterClosingLogicalFile = shouldSyncAfterClosingLogicalFile(checkpointBaseDir); + // Initialize the managed exclusive path using id as the child path name. + // Currently, we use the task-owned directory to place the merged private state. According + // to the FLIP-306, we later consider move these files to the new introduced + // task-manager-owned directory. + Path managedExclusivePath = new Path(taskOwnedStateDir, id); + createManagedDirectory(managedExclusivePath); + this.managedExclusiveStateDir = managedExclusivePath; + } + + @Override + public void registerSubtaskForSharedStates(SubtaskKey subtaskKey) { + String managedDirName = subtaskKey.getManagedDirName(); + Path managedPath = new Path(sharedStateDir, managedDirName); + if (!managedSharedStateDir.containsKey(subtaskKey)) { + createManagedDirectory(managedPath); + managedSharedStateDir.put(subtaskKey, managedPath); + } + } + + // ------------------------------------------------------------------------ + // logical & physical file + // ------------------------------------------------------------------------ + + protected LogicalFile createLogicalFile( + @Nonnull PhysicalFile physicalFile, @Nonnull SubtaskKey subtaskKey) { + LogicalFileId fileID = LogicalFileId.generateRandomId(); + return new LogicalFile(fileID, physicalFile, subtaskKey); + } + + @Nonnull + protected PhysicalFile createPhysicalFile(SubtaskKey subtaskKey, CheckpointedStateScope scope) + throws IOException { + PhysicalFile result; + Exception latestException = null; + + Path dirPath = getManagedDir(subtaskKey, scope); + + if (dirPath == null) { + throw new IOException( + "Could not get " + + scope + + " path for subtask " + + subtaskKey + + ", the directory may have not been created."); + } + + for (int attempt = 0; attempt < 10; attempt++) { + try { + OutputStreamAndPath streamAndPath = + EntropyInjector.createEntropyAware( + fs, + createPhysicalFilePath(dirPath), + FileSystem.WriteMode.NO_OVERWRITE); + FSDataOutputStream outputStream = streamAndPath.stream(); + Path filePath = streamAndPath.path(); + result = new PhysicalFile(outputStream, filePath, this.physicalFileDeleter, scope); + updateFileCreationMetrics(filePath); + return result; + } catch (Exception e) { + latestException = e; + } + } + + throw new IOException( + "Could not open output stream for state file merging.", latestException); + } + + private void updateFileCreationMetrics(Path path) { + // TODO: FLINK- add io metrics + LOG.debug("Create a new physical file {} for checkpoint file merging.", path); + } + + protected Path createPhysicalFilePath(Path dirPath) { + // this must be called after initFileSystem() is called + // so the checkpoint directories must be not null if we reach here + final String fileName = UUID.randomUUID().toString(); + return new Path(dirPath, fileName); + } + + protected final void deletePhysicalFile(FSDataOutputStream outputStream, Path filePath) { + + if (outputStream != null) { + try { + outputStream.close(); + } catch (IOException e) { + LOG.warn("Fail to close output stream when deleting file: {}", filePath); + } + } + ioExecutor.execute( + () -> { + try { + fs.delete(filePath, false); + LOG.debug("Physical file deleted: {}.", filePath); + } catch (IOException e) { + LOG.warn("Fail to delete file: {}", filePath); + } + }); + } + + // ------------------------------------------------------------------------ + // abstract methods + // ------------------------------------------------------------------------ + + /** + * Get a reused physical file or create one. This will be called in stream creation logic. Review Comment: The output stream of checkpoint writing. This is a TODO for FLINK-32073, in which there will be a new introduced `CheckpointStreamFactory` call this method to create streams (and corresponding physical files). -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org