Hi all, thanks for the KIP Jiunn-Yang, and for the thorough discussion. The changes to the streams group RPCs / records look fine to me.
LB1: I'd like to understand how by_start_time interacts with Kafka Streams. auto.offset.reset only affects source topics (and repartition topics), since changelog and global state topics have their offset management hardcoded to "none" and are handled separately via StoreChangelogReader. So I believe by_start_time would work correctly at the StreamsConfig level without interfering with state restoration. However, Streams also has a public AutoOffsetReset class (in org.apache.kafka.streams) with factory methods (earliest(), latest(), none(), byDuration()) used for per-source-topic overrides via Consumed.withOffsetResetPolicy(). The KIP doesn't mention adding a corresponding byStartTime() factory method to this class. Should this be included in the KIP's public interface changes? Thanks, Lucas On Tue, Apr 7, 2026 at 5:35 PM Chia-Ping Tsai <[email protected]> wrote: > > hi Jiunn-Yang > > Thanks for updating the KIP. It looks good to me. If DJ has no more feedback, > it would be good to kick off the vote. I hope this can make it into 4.4.0. > > 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
