masteryhx commented on code in PR #24766: URL: https://github.com/apache/flink/pull/24766#discussion_r1597905886
########## docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md: ########## @@ -292,4 +292,25 @@ The final checkpoint would be triggered immediately after all operators have rea without waiting for periodic triggering, but the job will need to wait for this final checkpoint to be completed. +## Unify file merging mechanism for checkpoints + +Flink 1.20 introduces a unified file merging mechanism for checkpointing, which allows scattered small checkpoint files Review Comment: Is it a mvp or production ready in Flink 1.20 ? We'd better to describe it if it's a mvp. ########## docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md: ########## @@ -292,4 +292,25 @@ The final checkpoint would be triggered immediately after all operators have rea without waiting for periodic triggering, but the job will need to wait for this final checkpoint to be completed. +## Unify file merging mechanism for checkpoints + +Flink 1.20 introduces a unified file merging mechanism for checkpointing, which allows scattered small checkpoint files +to be written into a single file, reducing the number of file creations and file deletions, helping to alleviate the pressure +of file system metadata management and file flooding problem. The unified fie merging mechanism can be enabled by setting +the property `state.checkpoints.file-merging.enabled` to `true`. **Note** that enabling this mechanism may lead to space amplification, +that is, the actual occupation on the file system will be larger than the checkpoint size. `state.checkpoints.file-merging.max-space-amplification` Review Comment: IIUC, the metric of checkpoint size should be consistent, right ? If It's compared with before or acutal state, let's adjust it as `before checkpoint size` or `actual state size`. ########## docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md: ########## @@ -292,4 +292,25 @@ The final checkpoint would be triggered immediately after all operators have rea without waiting for periodic triggering, but the job will need to wait for this final checkpoint to be completed. +## Unify file merging mechanism for checkpoints + +Flink 1.20 introduces a unified file merging mechanism for checkpointing, which allows scattered small checkpoint files +to be written into a single file, reducing the number of file creations and file deletions, helping to alleviate the pressure +of file system metadata management and file flooding problem. The unified fie merging mechanism can be enabled by setting +the property `state.checkpoints.file-merging.enabled` to `true`. **Note** that enabling this mechanism may lead to space amplification, +that is, the actual occupation on the file system will be larger than the checkpoint size. `state.checkpoints.file-merging.max-space-amplification` +can be used to limit the upper bound of space amplification. + +This mechanism is applicable to all types of state files in Flink, including keyed state, operator state, channel state +and changelog DSTL files. Subtask level granular merging is provided for shared scope state; TaskManager-level granular merging Review Comment: some unsupported types, e.g. DSTL files could be removed for now or listed as a limitation ? ########## docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md: ########## @@ -250,5 +250,20 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm 需要注意的是,这一行为可能会延长任务运行的时间,如果 checkpoint 周期比较大,这一延迟会非常明显。 极端情况下,如果 checkpoint 的周期被设置为 `Long.MAX_VALUE`,那么任务永远不会结束,因为下一次 checkpoint 不会进行。 +## 统一的 checkpoint 文件合并机制 + +Flink 1.20 引入了统一的 checkpoint 文件合并机制,该机制允许把分散的 checkpoint 文件写到同一个文件中,减少 checkpoint 文件创建删除的次数, +有助于减轻文件系统元数据管理的压力、 解决文件洪泛问题。可以通过将 `state.checkpoints.file-merging.enabled` 设置为 `true` 来开启该机制。 Review Comment: `文件洪泛问题` seems not a common description in chinese. How about just describing it more 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