Hi Apoorv,

Thanks for the review.

AM1, AM3: Yes, I agree. I have added OffsetResetStarategy class changes to KIP

AM2: Yes, I agree. I have updated the KIP to use ISO format

AM4: With ISO format, it's more flexible now. If required, we can
easily add support for reset to a particular datetime( similar
consumer group command).


Thanks,


On Mon, Nov 4, 2024 at 9:11 PM Apoorv Mittal <apoorvmitta...@gmail.com> 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
> >

Reply via email to