Hey Jiunn, Glad to see some progress around this issue.
I had a similar thought to David, that if the time is only known client side there are still edge cases for data loss. One case is static membership where from the perspective of a client they are free to restart their consumer task without actually having left or meaningfully affected the group. However, I think with the proposed implementation the timestamp is still reset here. So if the restart happens just after a partition is added and published to, but before the consumer metadata refreshed, the group still runs the risk of data loss. It could be argued that keeping the group 'stable' is a requirement for this feature to work, but sometimes it's not possible to accomplish. From: [email protected] At: 03/05/26 14:32:20 UTC-5:00To: [email protected] Subject: Re: [DISCUSS] KIP-1282: Prevent data loss during partition expansion for dynamically added partitions Hi Jiunn, Thanks for the KIP! I was also considering this solution while we discussed in the jira. It seems to work in most of the cases but not in all. For instance, let’s imagine a partition created just before a new consumer joins or rejoins the group and this consumer gets the new partition. In this case, the consumer will have a start time which is older than the partition creation time. This could also happen with the truncation case. It makes the behavior kind of unpredictable again. Instead of relying on a local timestamp, one idea would to rely on a timestamp provided by the server. For instance, we could define the time since the group became non-empty. This would define the subscription time for the consumer group. The downside is that it only works if the consumer is part of a group. In your missing semantic section, I don’t fully understand how the 4th point is improved by the KIP. It says start from earliest but with the change it would start from application start time. Could you elaborate? Best, David Le jeu. 5 mars 2026 à 12:47, 黃竣陽 <[email protected]> a écrit : > Hello chia, > > Thanks for your feedback, I have updated the KIP. > > Best Regards, > Jiunn-Yang > > > Chia-Ping Tsai <[email protected]> 於 2026年3月5日 晚上7:24 寫道: > > > > hi Jiunn > > > > chia_00: Would you mind mentioning KAFKA-19236 in the KIP? It would be > helpful to let readers know that "Dynamically at partition discovery" is > being tracked as a separate issue > > > > Best, > > Chia-Ping > > > > On 2026/03/05 11:14:31 黃竣陽 wrote: > >> Hello everyone, > >> > >> I would like to start a discussion on KIP-1282: Prevent data loss > during partition expansion for dynamically added partitions > >> <https://cwiki.apache.org/confluence/x/mIY8G> > >> > >> This proposal aims to introduce a new auto.offset.reset policy > by_start_time, anchoring the > >> offset reset to the consumer's startup timestamp rather than partition > discovery time, to prevent > >> silent data loss during partition expansion. > >> > >> Best regards, > >> Jiunn-Yang > >
