Hi, Jiunn-Yang, Thanks for the KIP.
The main motivation of the KIP is for Scenario 1: Partition Expansion Data Loss. I am wondering if this case can be covered by the existing by_duration. With KIP-848, if a new partition is added, the consumer client can pick it up after group.consumer.heartbeat.interval.ms, which defaults to 5 seconds. If the consumer sets auto.offset.reset to by_duration:5 secs, it won't miss any new messages on new partitions. It does mean that the consumer needs to pick up an extra 5 seconds' worth of data on the first start, but it probably doesn't make a difference. For Scenario 2: Log Truncation Data Loss, the to_start_time semantic seems weird to me. A common scenario is that a consumer application has been stopped for longer than the data retention time and the committed offsets are out of range. A consumer in this state doesn't seem much different from a consumer being started for the first time. It seems inconsistent that we use the latest offset for consumption in one case and the earliest in another. Also, a group is deleted after the consumer has been idle longer than offsets.retention.minutes. What's the semantic of to_start_time if the group creation time is unavailable? Jun On Thu, Apr 16, 2026 at 4:47 AM 黃竣陽 <[email protected]> wrote: > Hello Matthias > > Thanks for your feedback, > > MJS1: I have updated the KIP and added a startup check for Kafka Streams. > When to_start_time is configured and the Streams protocol is not enabled, > the > application will throw a ConfigException at startup, rather than surfacing > runtime > errors in the background StreamsThread. > > MJS2: I have bumped the RPC version in the KIP and will also keep an eye > on > KIP-1071 to avoid version conflicts. > > MJS3: --to-start-time uses AdminClient throughout the entire flow: > describeConsumerGroups() > / describeStreamsGroups() to fetch the group creation timestamp, > ListOffsets to resolve offsets, > and alterConsumerGroupOffsets() / alterStreamsGroupOffsets() to commit > offsets. None of these > operations go through the consumer protocol directly. So in my opinion, it > won't be affected. > > Best Regards, > Jiunn-Yang > > > Matthias J. Sax <[email protected]> 於 2026年4月16日 上午9:47 寫道: > > > > Just catching up on this KIP. Couple of questions/comments: > > > > > > MJS1: For Kafka Streams, given that the new policy only works with > "streams" groups, should we add a check at startup if "streams" is actually > enabled and fail early if not (eg throw ConfigException)? -- At least for > the config bases setting this seems doable -- not sure how easy it would be > if user set "to_start_time" programmatically. But maybe also worth to > explore? It's much better UX to fail early IMHO. > > > > The KIP already mentions this for consumer case, but not for KS. Relying > on the underlying consumer mechanism would result in runtime errors on the > background `StreamsThread` leading to bad UX. > > > > > > MJS2: We recently found a bug in KIP-1071 which requires us to bump the > heartbeat request/response version, too. There is already a draft PR that I > hope to get merged this or next week: > https://github.com/apache/kafka/pull/21799 --- more like an FYI to avoid > confusion -- not sure if we should cross-reference both KIPs to each other > about it? I did not update KIP-1071 yet, but plan to so this before we can > merge the draft PR. > > > > > > MJS3: Recently, we had a discussion about > `bin/kafka-consumer-groups.sh`, and Andrew highlighted that this tool > always uses "classic" protocol to commit offsets (what is not a problem in > the strict sense, because "mixed" group are supported, as pointed out by > David J). So I am wondering if introducing the new `--to-start-time` flag > would cause issues? Maybe not, but worth to double check. > > > > > > -Matthias > > > > On 4/12/26 3:51 AM, Andrew Schofield wrote: > >> Hi Jiunn-Yang, > >> Absolutely no need to apologise for non-native English :) There were > many review cycles before it was evident that to_ was slightly better than > by_. > >> The KIP looks great now. > >> Thanks, > >> Andrew > >> On 2026/04/11 08:57:37 黃竣陽 wrote: > >>> Hello Andrew, > >>> > >>> Apologies in advance for my non-native English — please bear with me :) > >>> > >>> Thanks for the clarification on AS13. After thinking it through, I > agree that > >>> to_start_time is a better name than by_start_time. Your reasoning is > convincing: > >>> by_duration computes a relative time (now - duration), while the new > policy > >>> anchors to an absolute, server-recorded timestamp (the group creation > time). > >>> The to_* prefix correctly conveys “seek to a fixed point”. > >>> > >>> Best Regards, > >>> Jiunn-Yang > >>> > >>>> Andrew Schofield <[email protected]> 於 2026年4月11日 下午4:20 寫道: > >>>> > >>>> Hi Jiunn-Yang, > >>>> Thanks for the updates. > >>>> > >>>> AS12: Thanks. > >>>> > >>>> AS13: I did actually mean to change all of the by_start_time, > byStartTime and --by-start-time variants across the entire KIP to > to_start_time, toStartTime and --to-start-time. I did say it was quite > pervasive, which is why I said you could ignore the suggestion :) > >>>> > >>>> AS14: Thanks. > >>>> > >>>> Thanks, > >>>> Andrew > >>>> > >>>> On 2026/04/11 02:13:19 黃竣陽 wrote: > >>>>> Hello Andrew, > >>>>> > >>>>> Thanks for the helpful feedback. > >>>>> > >>>>> AS12: GROUP-CREATION-TIME is now only shown in --describe --state > —verbose > >>>>> output. This avoids cluttering the default output for normal users. > >>>>> > >>>>> AS13: Addressed in the KIP. > >>>>> > >>>>> AS14: Applied the same changes to kafka-streams-groups.sh. Both > --describe --state —verbose > >>>>> and --reset-offsets --to-start-time are now supported for streams > groups as well. > >>>>> > >>>>> Best Regards, > >>>>> Jiunn-Yang > >>>>> > >>>>>> Andrew Schofield <[email protected]> 於 2026年4月11日 清晨5:17 寫道: > >>>>>> > >>>>>> A few more comments I'm afraid. > >>>>> > >>>>> > >>> > >>> > > > >
