Hi Manikumar, Thanks for the KIP. AS1: As Apoorv points out, the values of auto.offset.reset correspond to the enum OffsetResetStrategy. I would say that's a useful thing to maintain and I suggest adding a fourth value "by.duration" (BY_DURATION), and then another configuration for the duration "auto.offset.reset.by.duration". This would correspond with kafka-consumer-groups.sh --reset-offsets --by-duration.
I don't think that --reset-offsets --shift-by or --to-datetime really work as well for a config-based solution, and are much easier for interactive use with kafka-consumer-groups.sh. AS2: Typically, durations in client configs are specified as milliseconds, but this is unlikely to be that useable in this case where I expect the time period is more likely in the scale of hours or days. So, I'd like to reject your suggestion of a rejected alternative for ISO-8601. How about this? "auto.offset.reset.by.duration=P1D" is 1 day. "auto.offset.reset.by.duration=PT3H" is 3 hours. "auto.offset.reset.by.duration=PT1M" is 1 minute. Not that hard. The validation for this config could parse it into a Duration to make sure it is well formed. AS3: I like Matthias's suggestion about "latest-local". That seems very sensible in the context of tiered storage. Thanks, Andrew ________________________________________ From: Matthias J. Sax <mj...@apache.org> Sent: 05 November 2024 01:55 To: dev@kafka.apache.org <dev@kafka.apache.org> Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients Thanks for the KIP. A somewhat orthogonal question I have is, if we should try to merge this KIP with the existing KIP-842: https://cwiki.apache.org/confluence/display/KAFKA/KIP-842%3A+Add+richer+group+offset+reset+mechanisms While KIP-842 has a different intention, ie, separating the case of "no offset during initial startup" vs "offset out of bounds during processing", it might still be worth it? My motivation for this question is, that `auto.offset.reset` is one of the most central configs we have, and thus, doing a single larger change (ie, both KIPs together) might be better (less noise) than doing two independent changes? Thoughts? I know it might be a little bit of a stretch, but asking cannot hurt :) About the KIP itself, did we consider to add something like "latest-local" to say reset to oldest local, but not fetch from tiered storage? For the format of the config, did we consider what we do in `bin/kafka-consumer-group.sh`? It has multiple different options like --by-duration <duration: format `PnDTnHnMnS`> --shift-by <number-of-records> --to-datetime <fixed point in time: format `YYYY-MM-DDTHH:mm:SS.sss`> --to-offset <absolute offset> As it also uses ISO format, it might be good to also use them (even if I can see the appeal and simplicity of what you proposed). I also want to add to (AM1), as there is also `Topology.AutoOffsetReset` enum in Kafka Streams. I think we would need to convert this into a class. While it could be done is a follow up KIP, too, it seems best to do this holistically in a single KIP, because KS is not something on-top of Kafka, but it's part of Kafka. I am happy to help with the design and even PRs if necessary, but would strongly prefer to do it all in a single KIP. Btw: if we add something like "latest-local", it might also be good to extend `bin/kafka-consumer-group.sh` accordingly (even if the tool works slightly different, as it does commit an offset and there could be some race condition between committing "latest-local", tiering, and when the consumer is actually started? -Matthias On 11/4/24 7:38 AM, Apoorv Mittal wrote: > Hi Manikumar, > Thanks for the KIP, this new strategy would be helpful in specifying fetch > behaviour. > > AM1: The config `auto.offset.reset` is currently applied as per the enum > class OffsetResetStarategy which is part of kafka-clients javadoc > <https://javadoc.io/doc/org.apache.kafka/kafka-clients/latest/index.html>. > Are we also proposing to somehow add new definitions in the same class? > However as new configurations will be a string representation hence are we > moving away from OffsetResetStartegy enum class altogether? Should we > include the change in the KIP as OffsetResetStarategy is part of public > javadoc? > > AM2: While I can see the ISO-8601 format is in the rejected alternative, > should we not follow some standard of defining duration which has already > been adopted in other systems? > > AM3: We've introduced new config values using the format minus-n-hours, > minus-n-days, minus-n-months, and minus-n-years. Should we explicitly > define the "minus" prefix, or is it implied? > > AM4: When supporting duration-based resets, should we also allow users to > specify a specific checkpoint time? For example, if a checkpoint occurs 2 > days, 5 hours and 30 minutes earlier, the current four formats > (minus-n-hours, minus-n-days, minus-n-months, minus-n-years) might not be > sufficient. Should we consider adding a format to accommodate specific > checkpoint times, or is there a reason to limit the supported formats? > > > Regards, > Apoorv Mittal > > > On Mon, Nov 4, 2024 at 9:23 AM Manikumar <manikumar.re...@gmail.com> wrote: > >> Hi everyone, >> I would like to start a discussion on KIP-1106: >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1106%3A+Add+duration+based+offset+reset+option+for+consumer+clients >> >> This KIP proposes to add an additional auto offset reset strategy for >> consumer clients. >> >> Regards, >> Manikumar >> >