curcur commented on a change in pull request #18431: URL: https://github.com/apache/flink/pull/18431#discussion_r793315064
########## File path: docs/content/docs/ops/state/state_backends.md ########## @@ -325,6 +325,126 @@ public class MyOptionsFactory implements ConfigurableRocksDBOptionsFactory { {{< top >}} +## Enabling Changelog + +// todo: Chinese version of all changed docs + +// todo: mention in [large state tuning]({{< ref "docs/ops/state/large_state_tuning" >}})? or 1.16? + +{{< hint warning >}} The feature is in experimental status. {{< /hint >}} + +{{< hint warning >}} Enabling Changelog may have a negative performance impact on your application (see below). {{< /hint >}} + +### Introduction + +Changelog is a feature that aims to decrease checkpointing time, and therefore end-to-end latency in exactly-once mode. + +Most commonly, checkpoint duration is affected by: + +1. Barrier travel time and alignment, addressed by + [Unaligned checkpoints]({{< ref "docs/ops/state/checkpointing_under_backpressure#unaligned-checkpoints" >}}) + and [Buffer debloating]({{< ref "docs/ops/state/checkpointing_under_backpressure#buffer-debloating" >}}) +2. Snapshot creation time (so-called synchronous phase), addressed by Asynchronous snapshots +3. Snapshot upload time (asynchronous phase) + +The latter (upload time) can be decreased by [Incremental checkpoints]({{< ref "#incremental-checkpoints" >}}). However, +even with Incremental checkpoints, large deployments tend to have at least one task in every checkpoint that uploads a +lot of data (e.g. after compaction). + +With Changelog enabled, Flink uploads state changes continuously, forming a changelog. On checkpoint, only the relevant +part of this changelog needs to be uploaded. Independently, configured state backend is checkpointed in the +background periodically. Upon successful upload, changelog is truncated. + +As a result, asynchronous phase is reduced, as well as synchronous phase (in particular, long-tail). + +On the flip side, resource usage is higher: + +- more files are created on DFS +- more IO bandwidth is used to upload +- more CPU used to serialize state changes +- more memory used by Task Managers to buffer state changes +- todo: more details after testing, maybe link to blogpost + +Recovery time is another thing to consider. Depending on the `state.backend.changelog.periodic-materialize.interval`, +changelog can become lengthy and replaying it may take more time. However, recovery time combined with checkpoint +duration will likely be still lower than in non-changelog setup, providing lower end-to-end latency even in failover +case. Review comment: What I want to say is there is no evidence showing that recovery time is guaranteed to be reduced. 1. "less data replay" has an assumption that checkpoint duration is reduced (there is not always a significant reduction as I mentioned above). 2. Replaying of changelog to DB is an additional cost for recovery with changelog vs. normal recovery. 1+2 made me not be convinced with "However, recovery time combined with checkpoint duration will likely be still lower than in non-changelog setup, providing lower end-to-end latency even in failover case." -- 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