1996fanrui commented on code in PR #24653: URL: https://github.com/apache/flink/pull/24653#discussion_r1594875205
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageAccess.java: ########## @@ -232,7 +232,9 @@ public FsMergingCheckpointStorageAccess toFileMergingStorage( FileMergingSnapshotManager mergingSnapshotManager, Environment environment) throws IOException { return new FsMergingCheckpointStorageAccess( - fileSystem, + /* Multiple subtask/threads would share one output stream, + SafetyNetWrapperFileSystem cannot be used to prevent different threads from interfering with each other when exiting. */ + FileSystem.getUnguardedFileSystem(checkpointsDirectory.toUri()), Review Comment: I'm thinking could `checkpointsDirectory` as the parameter of FsMergingCheckpointStorageAccess constructor?And we initialize the fileSystem inside of FsMergingCheckpointStorageAccess constructor. It can avoid all callers pass a GuardedFileSystem. WDYT? ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java: ########## @@ -483,27 +485,42 @@ protected StreamTask( } this.systemTimerService = createTimerService("System Time Trigger for " + getName()); - + final CheckpointStorageAccess finalCheckpointStorageAccess = checkpointStorageAccess; + + ChannelStateWriter channelStateWriter = + configuration.isUnalignedCheckpointsEnabled() + ? openChannelStateWriter( + getName(), + () -> { + if (finalCheckpointStorageAccess + instanceof FsMergingCheckpointStorageAccess) { + return finalCheckpointStorageAccess; + } else { + return checkpointStorage.createCheckpointStorage( Review Comment: Would you mind adding some comments here to explain why we don't pass `CheckpointStorageAccess` directly? -- 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