Hexiaoqiao commented on code in PR #22590: URL: https://github.com/apache/flink/pull/22590#discussion_r1203358593
########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java: ########## @@ -0,0 +1,285 @@ +/* + * 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.FileSystem; +import org.apache.flink.core.fs.OutputStreamAndPath; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingCheckpointUtils.SnapshotFileSystemInfo; +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.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(SnapshotFileSystemInfo fileSystemInfo) throws IOException { + initFileSystem( + fileSystemInfo.fs, + fileSystemInfo.checkpointBaseDirectory, + fileSystemInfo.sharedStateDirectory, + fileSystemInfo.taskOwnedStateDirectory); + } + + @Override + public void addSubtask(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. + * + * @param subtaskKey the subtask key for the caller + * @param checkpointId the checkpoint id + * @param scope checkpoint scope + * @return the requested physical file. + * @throws IOException thrown if anything goes wrong with file system. + */ + @Nonnull + protected abstract PhysicalFile getOrCreatePhysicalFileForCheckpoint( + SubtaskKey subtaskKey, long checkpointId, CheckpointedStateScope scope) + throws IOException; + + /** + * Try to return an existing physical file to the manager for next reuse. Delete if needed. + * + * @param subtaskKey the subtask key for the caller + * @param checkpointId in which checkpoint this physical is requested. + * @param physicalFile the returning checkpoint + * @throws IOException thrown if anything goes wrong with file system. + */ + protected abstract void tryReusingPhysicalFile( + SubtaskKey subtaskKey, long checkpointId, PhysicalFile physicalFile) throws IOException; + + // ------------------------------------------------------------------------ + // file system + // ------------------------------------------------------------------------ + + /** + * Initiate the {@link FileMergingSnapshotManager} so that it can interact with the file + * systems, e.g. recording path information, creating working directories, etc. + * + * @param fileSystem The filesystem to write to. + * @param checkpointBaseDir The base directory for checkpoints. + * @param sharedStateDir The directory for shared checkpoint data. + * @param taskOwnedStateDir The name of the directory for state not owned/released by the + * master, but by the TaskManagers. + * @throws IOException Thrown, if the working directory cannot be created. + */ + private void initFileSystem( + FileSystem fileSystem, + Path checkpointBaseDir, + Path sharedStateDir, + Path taskOwnedStateDir) + throws IOException { + 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 = checkpointBaseDir; + this.sharedStateDir = sharedStateDir; + this.taskOwnedStateDir = 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 Path getManagedDir(SubtaskKey subtaskKey, CheckpointedStateScope scope) { + if (scope.equals(CheckpointedStateScope.SHARED)) { + return managedSharedStateDir.get(subtaskKey); + } else { + return managedExclusiveStateDir; + } + } + + static boolean shouldSyncAfterClosingLogicalFile(Path checkpointDir) { + // Currently, we do file sync regardless of the file system. + return true; + } + + // ------------------------------------------------------------------------ + // utilities + // ------------------------------------------------------------------------ + + private void createManagedDirectory(Path managedPath) { + try { + if (!fs.exists(managedPath)) { Review Comment: Here `fs.exists()` and `fs.getFileStatus()` is actually the same method at the underly layer, we could merge them and reduce RPC requests (such as to HDFS), And `isDir` is one deprecated interface, we should switch to `isDirectory`. Would something like this work? (NOTE: I have not tested this at all) ``` boolean isPathExist = false; boolean isDirectory = false; try { FileStatus status = fs.getFileStaus(managedPath); isPathExist = true; if (status.isDirectory()) { isDirectory = true; } } catch (FileNotFoundException e) { isPathExist = false; } if (!isPathExist) { fs.mkdirs(managedPath); } else if (!isDirectory) { throw new FlinkRuntimeException( "The managed path " + managedPath + " for file-merging is occupied by another file. Cannot create directory."); } ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingCheckpointUtils.java: ########## @@ -0,0 +1,79 @@ +/* + * 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.FileSystem; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; + +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +/** Utilities for file-merging checkpoints. */ +public class FileMergingCheckpointUtils { + + // ------------------------------------------------------------------------ + // FileMergingSnapshotManager initialization related + // ------------------------------------------------------------------------ + + /** A class that packs the file system info for snapshot. */ + public static class SnapshotFileSystemInfo { + FileSystem fs; + Path checkpointBaseDirectory; + Path sharedStateDirectory; + Path taskOwnedStateDirectory; + + public SnapshotFileSystemInfo( + FileSystem fs, + Path checkpointBaseDirectory, + @Nullable Path sharedStateDirectory, + Path taskOwnedStateDirectory) { + this.fs = fs; + this.checkpointBaseDirectory = checkpointBaseDirectory; Review Comment: Need to check if `checkpointBaseDirectory` is prefix of `sharedStateDirectory` and `taskOwnedStateDirectory` here? TBH I am not familiar with it if we plan to relax this restricts. ########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/WithinCheckpointFileMergingSnapshotManager.java: ########## @@ -0,0 +1,97 @@ +/* + * 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.api.java.tuple.Tuple3; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.runtime.state.CheckpointedStateScope; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executor; + +/** A {@link FileMergingSnapshotManager} that merging files within a checkpoint. */ +public class WithinCheckpointFileMergingSnapshotManager extends FileMergingSnapshotManagerBase { + + /** A dummy subtask key to reuse files among subtasks for private states. */ + private static final SubtaskKey dummySubtaskKey = new SubtaskKey("dummy", -1, -1); + + /** + * OutputStreams to be reused when writing checkpoint files. For WITHIN_BOUNDARY mode, physical + * files are NOT shared among multiple checkpoints. + */ + private final Map<Tuple3<Long, SubtaskKey, CheckpointedStateScope>, PhysicalFile> fileMap; + + public WithinCheckpointFileMergingSnapshotManager(String id, Executor ioExecutor) { + // currently there is no file size limit For WITHIN_BOUNDARY mode + super(id, ioExecutor); + fileMap = new HashMap<>(); + } + + @Override + @Nonnull + protected PhysicalFile getOrCreatePhysicalFileForCheckpoint( + SubtaskKey subtaskKey, long checkpointId, CheckpointedStateScope scope) + throws IOException { + // TODO: FLINK-32076 will add a file pool for each subtask key. + Tuple3<Long, SubtaskKey, CheckpointedStateScope> fileKey = + Tuple3.of( + checkpointId, + scope == CheckpointedStateScope.SHARED ? subtaskKey : dummySubtaskKey, + scope); + PhysicalFile file; + synchronized (fileMap) { + file = fileMap.remove(fileKey); + if (file == null) { + file = createPhysicalFile(subtaskKey, scope); + } + } + return file; + } + + @Override + protected void tryReusingPhysicalFile( + SubtaskKey subtaskKey, long checkpointId, PhysicalFile physicalFile) + throws IOException { + // TODO: FLINK-32076 will add a file pool for reusing. + CheckpointedStateScope scope = physicalFile.getScope(); + Tuple3<Long, SubtaskKey, CheckpointedStateScope> fileKey = + Tuple3.of( + checkpointId, + scope == CheckpointedStateScope.SHARED ? subtaskKey : dummySubtaskKey, + scope); + PhysicalFile current; + synchronized (fileMap) { + current = fileMap.putIfAbsent(fileKey, physicalFile); + } + if (syncAfterClosingLogicalFile) { + FSDataOutputStream os = physicalFile.getOutputStream(); + if (os != null) { + os.flush(); Review Comment: I don't think we need invoke both `flush` and `sync` here. For HDFS, `sync` means flush out the data in client's buffer and persist data to the disk device all the way. which `flush` only flush out the data in client's user buffer. So I think it's enough to invoke `sync` only here. IIUC, it is similar to POSIX defines. Not sure if we have to do that for any other storages. -- 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