Makes total sense.

On 2026/04/03 12:25:57 Chia-Ping Tsai wrote:
> > AS7: I'll have one last try at persuasion here, but I expect I won't
> succeed. In KIP-1274, the default group protocol for KafkaConsumer is
> changing to "consumer". It is true that "classic" is still supported, but
> users will need to explicitly enable it by setting
> "group.protocol=classic". I think this means we could change the default
> for "auto.offset.reset" to "by_start_time" in AK 5.0. Whether we want to do
> it is a different matter.
> 
> I also love this new policy, but changing the default right now is a bit
> too eager for this KIP. I would prefer to implement it in 4.4.0 as an
> opt-in feature and announce it to users first. If we see good adoption and
> positive feedback from the community, we can then open a new KIP to discuss
> changing the default in a future release. Let's take it one step at a time
> 
> 
> Andrew Schofield <[email protected]> 於 2026年4月3日週五 下午7:58寫道:
> 
> > Hi Jiunn-Yang,
> > Thanks for your response.
> >
> > AS1-AS6: lgtm
> >
> > AS7: I'll have one last try at persuasion here, but I expect I won't
> > succeed. In KIP-1274, the default group protocol for KafkaConsumer is
> > changing to "consumer". It is true that "classic" is still supported, but
> > users will need to explicitly enable it by setting
> > "group.protocol=classic". I think this means we could change the default
> > for "auto.offset.reset" to "by_start_time" in AK 5.0. Whether we want to do
> > it is a different matter.
> >
> > AS8, AS9: lgtm
> >
> > AS10: I'm not a fan of the "Epoch-ms" wording, particularly in the Admin
> > API javadoc. I would just use "The creation time of the group." or similar.
> >
> > AS11: I suspect you need to change StreamsGroupHeartbeatResponse too.
> > Probably need a Kafka Streams expert to weigh in how
> > `auto.offset.reset=by_start_time` would work for the consumer in a streams
> > group.
> >
> > Thanks,
> > Andrew
> >
> > On 2026/04/03 01:18:02 黃竣陽 wrote:
> > > Hello all,
> > >
> > > AS4
> > >
> > > I agree that bumping the version for request/response protocols provides
> > a
> > > meaningful benefit: it allows the consumer to distinguish between "the
> > broker
> > > does not support this feature" and "the broker supports it, but the
> > group has no
> > > creation time." This enables more actionable error messages.
> > >
> > > For __consumer_offsets records (ConsumerGroupMetadataValue and
> > > StreamsGroupMetadataValue), I will keep tagged fields. These records
> > have no version
> > > negotiation. They are written and read internally by the broker. Tagged
> > fields allow old brokers
> > > to safely ignore unknown tags during log replay, and avoid the need to
> > rewrite existing records
> > > on upgrade.
> > >
> > > Best Regards,
> > > Jiunn-Yang
> > >
> > > > Chia-Ping Tsai <[email protected]> 於 2026年4月3日 上午8:01 寫道:
> > > >
> > > > AS4
> > > >
> > > > Bumping the version actually provides a significant benefit. It allows
> > the new consumer to offer more actionable error messages to users, because
> > it can clearly distinguish between "the broker is too old to support this
> > feature" and "the broker supports it, but the group has no creation time."
> > > >
> > > > Furthermore, by bumping the version, the auto-generated protocol code
> > can gracefully handle the serialization (e.g., ignoring the field if the
> > client/broker is communicating with an older version).
> > > >
> > > > On 2026/04/02 15:04:39 黃竣陽 wrote:
> > > >> Hi Andrew,
> > > >>
> > > >> Thanks for the thorough review.
> > > >>
> > > >> AS1:
> > > >> The updated KIP now explicitly addresses these scenarios:
> > > >> - Configuring auto.offset.reset=by_start_time with a classic consumer
> > group throws
> > > >> ConfigException at startup.
> > > >> - Classic groups upgraded to modern groups cannot use by_start_time
> > immediately,
> > > >> because the group creation timestamp was not recorded at the time the
> > group was originally
> > > >> created. The consumer throws `GroupCreationTimeNotAvailableException`
> > in this case.
> > > >> Users must delete and recreate the group to obtain a fresh creation
> > timestamp.
> > > >>
> > > >> AS2, AS3, AS5: Updated the KIP accordingly.
> > > >>
> > > >> AS4:
> > > >>
> > > >> Tagged fields fit well because GroupCreationTimeMs / CreationTimeMs
> > is optional,
> > > >> with a default value of -1 ("unknown") that has clear semantics,
> > especially for pre-existing
> > > >> groups. They are only serialized when non-default, so they add no
> > overhead. Older
> > > >> brokers simply ignore unknown tags when replaying __consumer_offsets,
> > requiring no
> > > >> record rewriting during upgrades.
> > > >>
> > > >> In contrast, a version bump would require expanding validVersions,
> > adding version checks
> > > >> across read/write paths, and provides no real benefit given the
> > field’s optional nature and
> > > >> well-defined default. This follows Kafka’s convention: use tagged
> > fields for optional additions
> > > >> with meaningful defaults, and version bumps for mandatory or semantic
> > changes.
> > > >>
> > > >> AS6:
> > > >>
> > > >> Thanks for the clarification. I’ve incorporated this into the KIP.
> > > >>
> > > >> AS7:
> > > >>
> > > >> I agree that by_start_time better matches user expectations for the
> > default offset reset policy.
> > > >> However, since it is not supported by classic consumer groups and
> > would raise a ConfigException
> > > >> at startup, changing the default is only safe after the classic
> > protocol is removed from KafkaConsumer.
> > > >> Per KIP-1274, classic protocol support will be deprecated in Kafka
> > 6.0. After that, the
> > > >> default can be changed from latest to by_start_time.
> > > >>
> > > >> AS8:
> > > >>
> > > >> InvalidOffsetException is an abstract class that requires subclasses
> > to implement partitions(),
> > > >> implying the error is tied to specific partitions with invalid
> > offsets.
> > > >>
> > > >> GroupCreationTimeNotAvailableException does not fit this model. The
> > root cause is a missing
> > > >> group-level timestamp, not an issue with any particular partition.
> > Implementing partitions()
> > > >> would require returning an empty or artificial set, which violates
> > the API’s semantics.
> > > >>
> > > >> Therefore, I believe extending KafkaException is the better choice.
> > > >>
> > > >> AS9:
> > > >>
> > > >> Renamed to GroupCreationTimeNotAvailableException throughout the KIP.
> > This better
> > > >> describes the state ("not available") rather than the cause.
> > > >>
> > > >> Happy to hear any further feedback or suggestions.
> > > >>
> > > >> Best Regards,
> > > >> Jiunn-Yang
> > > >>
> > > >>
> > > >>> Andrew Schofield <[email protected]> 於 2026年4月2日 晚上8:34 寫道:
> > > >>>
> > > >>> Hi Jiunn-Yang,
> > > >>> Thanks for this great KIP. Some comments.
> > > >>>
> > > >>> AS1: A limitation of this KIP is that it will not support classic
> > consumer groups. I think this is not a problem and it will be a motivator
> > for users to adopt modern consumer groups, but it will be necessary to test
> > attempting to use the new policy with classic consumer groups and make sure
> > that the ConsumerGroupDescription lacks the creation timestamp and so on.
> > There is also the question of classic groups which are upgraded to modern
> > consumer groups.
> > > >>> AS2: It seems to me that you should also support streams groups,
> > both in a StreamsGroupMetadataValue and StreamsGroupDescription.
> > > >>> AS3: What you are describing as metadata log changes are actually
> > records on the __consumer_offsets topic.
> > > >>> AS4: While I understand the need to use tagged fields in the
> > records, I do not see why you cannot bump the versions for
> > ConsumerGroupHeartbeat and ConsumerGroupDescribe in order to support the
> > new fields.
> > > >>> AS5: In the Admin API, we already have an optional timestamp in
> > TransactionDescription.transactionStartTimeMs which uses the type
> > OptionalLong. Any reason not to use the same in ConsumerGroupDescription?
> > > >>> AS6: I don't think share group support is needed. "latest" already
> > starts at offset 0 for newly added partitions for share groups.
> > > >>> AS7: Might you consider making "by_start_time" the default value for
> > the auto.offset.reset consumer config in the next major release? I know
> > that's a bold move, but I think what you're getting with the new option is
> > what people expected "latest" to behave like.
> > > >>> AS8: I wonder whether GroupCreationTimeUnknownException should
> > extend the abstract InvalidOffsetException.
> > > >>> AS9: To follow existing exception names, I think
> > UnknownGroupCreationTimeException, GroupCreationTimeNotAvailableException
> > (my favourite) or GroupCreationTimeNotFoundException would better match
> > what we already have. wdyt?
> > > >>>
> > > >>> Thanks,
> > > >>> Andrew
> > > >>>
> > > >>> On 2026/04/02 11:34:10 黃竣陽 wrote:
> > > >>>> Hi chia,
> > > >>>>
> > > >>>> chia_13:
> > > >>>> `ConsumerGroupDescription` now gains a new method
> > `groupCreationTimeMs()`:
> > > >>>>
> > > >>>> `groupCreationTimeMs()` (`Optional<Long>`): the epoch-ms when this
> > consumer group
> > > >>>> was first created on the broker. `Optional.empty()` if unknown.
> > > >>>>
> > > >>>> Please take a look at the updated KIP and let us know if you have
> > further questions.
> > > >>>>
> > > >>>> Best Regards,
> > > >>>> Jiunn-Yang
> > > >>>>
> > > >>>>> Chia-Ping Tsai <[email protected]> 於 2026年4月2日 上午9:58 寫道:
> > > >>>>>
> > > >>>>> hi Jiunn
> > > >>>>>
> > > >>>>> chia_13: should we update the public API: ConsumerGroupDescription
> > as well?
> > > >>>>>
> > > >>>>> Best,
> > > >>>>> Chia-Ping
> > > >>>>>
> > > >>>>> On 2026/04/01 11:59:46 黃竣陽 wrote:
> > > >>>>>> Hi chia,
> > > >>>>>>
> > > >>>>>> Thank you for the thoughtful feedback!
> > > >>>>>>
> > > >>>>>> chia_10: I have added a clarification in the Proposed Changes
> > section to address this. In this case,
> > > >>>>>> ListOffsetsRequest will resolve to the Log End Offset, but this
> > is not a fallback to `latest`. The distinction
> > > >>>>>> matters: `latest` is a direct seek to the Log End Offset, whereas
> > `by_start_time` always anchors to the
> > > >>>>>> group creation timestamp and lets the lookup result follow
> > naturally. The consumer will begin consuming
> > > >>>>>> any new records produced after this point normally.
> > > >>>>>>
> > > >>>>>> chia_11: I have introduced a new exception
> > `GroupCreationTimeUnknownException` to signal
> > > >>>>>> this specific condition.
> > > >>>>>>
> > > >>>>>> chia_12: I have updated the KIP to expose the group creation
> > timestamp via `AdminClient.describeConsumerGroups()`
> > > >>>>>> , accessible through
> > `ConsumerGroupDescription.groupCreationTimeMs()`. This is backed by a new
> > > >>>>>> `GroupCreationTimeMs` field in `ConsumerGroupDescribeResponse.
> > > >>>>>>
> > > >>>>>> Best Regards,
> > > >>>>>> Jiunn-Yang
> > > >>>>>>
> > > >>>>>>> Chia-Ping Tsai <[email protected]> 於 2026年4月1日 下午2:33 寫道:
> > > >>>>>>>
> > > >>>>>>> hi Jiunn
> > > >>>>>>>
> > > >>>>>>> thanks for updating KIP. I have a couple of questions.
> > > >>>>>>>
> > > >>>>>>> chia_10: What happen if the group creation time is strictly
> > greater than the timestamp of the latest record in the partition? Will the
> > consumer fall back to the "latest" offset behaviour in this case? It could
> > be good to elucidate that in the KIP
> > > >>>>>>>
> > > >>>>>>> chia_11: Could you specify which exact exception will be thrown
> > when the group creation time is unknown
> > > >>>>>>>
> > > >>>>>>> chia_12: Since the group creation time is now a critical
> > semantic anchor, should we also expose it via the `DescribeGroupsResponse`?
> > > >>>>>>>
> > > >>>>>>> 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
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>>
> > > >>
> > > >>
> > >
> > >
> >
> 

Reply via email to