Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-12-11 Thread Manikumar
; > > > >>> > > > > as `public` variables? Seems we don't need them, but that they >>> are >>> > > > > rather an internal implementation detail? >>> > > > > >>> > > > > In general, I would

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-19 Thread Manikumar
; > > > > have method signatures in the KIP. >> > > > > >> > > > > What I believe we need to add is a `protected` constructor (for >> the >> > > > > internal sub-class): >> > > > > >> > > > >

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-13 Thread Manikumar
or (for the > > > > > internal sub-class): > > > > > > > > > > protected AutoOffsetReset(AutoOffsetReset autoOffsetReset); > > > > > > > > > > > > > > > > > > > > > > > > > -Matthia

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-13 Thread Jun Rao
gt; > > On 11/12/24 6:09 AM, Apoorv Mittal wrote: > > > > > Thanks Manikumar for explaining, sounds good to me. > > > > > > > > > > Regards, > > > > > Apoorv Mittal > > > > > > > > > > > > > > > On Tue, Nov 12, 2024 at 1:47 PM Andrew Sch

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-13 Thread Manikumar
Regards, > > > > Apoorv Mittal > > > > > > > > > > > > On Tue, Nov 12, 2024 at 1:47 PM Andrew Schofield < > > > > andrew_schofield_j...@outlook.com> wrote: > > > > > > > >> Hi, > > > >> Looks

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-13 Thread Jun Rao
gt;> Hi, > > >> Looks good now. Just one suggestion. > > >> > > >> AS8: Instead of "back:30D", I wonder whether the word 'duration' ought > > to > > >> be > > >> used to be consistent with kafka-consumer-group

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Manikumar
groups.sh. So, > >> "by-duration:P3D" or "duration:P3D" might be better. > >> > >> The overall idea of merging the configs into one config is fine in the > >> current > >> text of the KIP. > >> > >> Thanks, > >&

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Matthias J. Sax
ion:P3D" or "duration:P3D" might be better. The overall idea of merging the configs into one config is fine in the current text of the KIP. Thanks, Andrew From: Manikumar Sent: 12 November 2024 13:30 To: dev@kafka.apache.org Subject: Re:

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Apoorv Mittal
e KIP. > > Thanks, > Andrew > > > From: Manikumar > Sent: 12 November 2024 13:30 > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option > for consumer clients > > Hi Apoorv, > &g

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Andrew Schofield
g the configs into one config is fine in the current text of the KIP. Thanks, Andrew From: Manikumar Sent: 12 November 2024 13:30 To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients Hi A

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Manikumar
Hi Apoorv, AM7: AutoOffsetReset.java is for Kafka Streams API. I am not proposing any public Interface/class for Kafka Consumer. As mentioned in the KIP, even though OffsetResetStrategy is a public class, it's not used in any public APIs. I think new internal classes should be sufficient. AM8: Fi

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Apoorv Mittal
Hi, I read the changes for single configuration and deprecated OffsetResetStrategy.java. AM7: Question: The KIP says that previous supported values were earliest/latest/none and new back: config would be added. We have no definition of "none" in the newly introduced AutoOffsetReset.java class henc

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-12 Thread Manikumar
Thanks Ismael and Lianet for the reviews. Based on suggestions, I have updated the KIP to in favour of having a single config (auto.offset.reset). I have also adopted the Lianet's suggestion on naming. auto.offset.reset=back:P3D -> reset back 3 days Let me know if there are any concerns. Than

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-09 Thread Lianet M.
Hi all. Thanks Manikumar for the nice improvement, useful indeed. I also lean towards having a single config given that it's all about the reset policy, seems all the same "what" (auto reset policy) and we are just extending the with a new behaviour. Regarding the naming (agree on duration being c

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Ismael Juma
Thanks for the KIP, this is useful. A comment below. On Thu, Nov 7, 2024 at 4:51 PM Matthias J. Sax wrote: > I am personally not convinced that adding a new config > `auto.offset.reset.by.duration` is the best way. Kafka in general has > way too many configs and trying to avoid adding more confi

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Jun Rao
> > From: Jun Rao > Sent: 08 November 2024 17:37 > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option > for consumer clients > > Hi, ManiKumar, > > Thanks for t

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Manikumar
at this is just another convenient way and not > >>>> necessary to support functionality. Skipping it sounds good to me. > >>>> > >>>> Regards, > >>>> Apoorv Mittal > >>>> > >>>> > >>>> On

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Andrew Schofield
November 2024 17:37 To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients Hi, ManiKumar, Thanks for the updated PR. Another comment. JR3. Is there any benefit of using ISO8601 for auto.offset.reset.by.duration instead of just ms in

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Matthias J. Sax
mar Sent: 06 November 2024 04:20 To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients Hi Apoorv, Thanks for the review. AM5: Thanks, Updated the KIP. AM6: Currently we can achieve this with the combination of offsetsForTimes() a

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Jun Rao
ook.com> wrote: > > > >> > > > >>> Hi Manikumar, > > > >>> > > > >>> AS7: In response to AM6, I agree that in general the combination > > > >>> of offsetsForTimes() and seek() will do what is

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-08 Thread Manikumar
> > >>> of offsetsForTimes() and seek() will do what is required. The one case > > >> that > > >>> I think is more tricky is seeking to the local earliest/beginning. > > >>> > > >>> The Consumer API has Consumer.beginningOffsets and > > >>> seekToBeginning, which corres

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-07 Thread Manikumar
find the local earliest > >>> offset. > >>> That can be found using Admin.listOffsets with OffsetSpec.EarliestLocal. > >>> We could add equivalents of Consumer.beginningOffsets and > >>> Consumer.seekToBeginning. An application could always use the >

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-07 Thread Manikumar
eginningOffsets and > > > Consumer.seekToBeginning. An application could always use the > > > Admin API with the Consumer API. > > > > > > Personally, I would not complicate this KIP with these new methods when > > > the focus is more on duration-based offset reset and configs. > > > >

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-07 Thread Matthias J. Sax
he Consumer API. Personally, I would not complicate this KIP with these new methods when the focus is more on duration-based offset reset and configs. Thanks, Andrew From: Manikumar Sent: 06 November 2024 04:20 To: dev@kafka.apache.org Subject: Re: [DIS

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-07 Thread Jun Rao
ould always use the > > Admin API with the Consumer API. > > > > Personally, I would not complicate this KIP with these new methods when > > the focus is more on duration-based offset reset and configs. > > > > Thanks, > > Andrew > > ____

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-06 Thread Andrew Schofield
ndrew From: Manikumar Sent: 06 November 2024 04:20 To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients Hi Apoorv, Thanks for the review. AM5: Thanks, Updated the KIP. AM6: Currently we can achieve this with the combinati

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-06 Thread Apoorv Mittal
November 2024 04:20 > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option > for consumer clients > > Hi Apoorv, > > Thanks for the review. > > AM5: Thanks, Updated the KIP. > > AM6: Currently we can ach

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Manikumar
Hi Apoorv, Thanks for the review. AM5: Thanks, Updated the KIP. AM6: Currently we can achieve this with the combination of offsetsForTimes() and seek() API.. Maybe we can add a generic API like seekToTimes(Map timestampsToSearch) if required. Let's see what others think. Thanks, On Wed, Nov 6

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Manikumar
aylight saving comment only apply to share groups? > If we need to say it (not sure whether we do), it needs to apply to both > consumer groups and share groups. > > Thanks, > Andrew > > > From: Manikumar > Sent: 05 November 2024

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Apoorv Mittal
Hi Manikumar, Thanks for the changes. Just minor comment and a question: AM5: The description "How to initialize the share-partition start offset:" for "*share.auto.offset.reset*" seems incomplete as it ends with `:`. Should we write the details as defined in KIP-932 i.e. How to initialize the sh

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Manikumar
Hi Matthias, Thanks for the review. MS1: looks like KIP-842 is stuck in the voting. Current changes should fit into KIP-842 (few updates definitely required) as KIP-842 is proposing to add new configs (auto.offset.reset.on.no.initial.offset and auto.offset.reset.on.invalid.offset). I prefer to re

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Andrew Schofield
ar Sent: 05 November 2024 17:09 To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients 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

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Manikumar
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( s

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Manikumar
Hi Dániel, Thanks for the review, DU1,DU2: Based on suggestions, I have updated the config option to support ISO format. DU3: We will compute the timestamp on the client side and use that timestamp to fetch the offset from the server. This will be similar to the kafka-consumer-group cmd's --by-

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Manikumar
> in the context of tiered storage. > > Thanks, > Andrew > ____________ > From: Matthias J. Sax > Sent: 05 November 2024 01:55 > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-1106: Add duration based offset reset option for > consumer cli

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-05 Thread Andrew Schofield
o 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 Sent: 05 November 2024 01:55 To: dev@kafka.apache.org Subje

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-04 Thread Matthias J. Sax
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

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-04 Thread Apoorv Mittal
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

Re: [DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-04 Thread Dániel Urbán
Hi Manikumar, Thanks for the KIP! I have a few questions: 1. Why did you choose the "minus-" prefix for the new value format? Can't we just drop the prefix? If we have to keep the prefix, I'd suggest "last-N-Duration", as it somewhat "rhymes" with latest-earliest. 2. Should minutes and seconds be

[DISCUSS] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-04 Thread Manikumar
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