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 confusing), what about something to show that it's simply about how far back to reset:
auto.offset.reset=EARLIEST auto.offset.reset=BACK:P3D -> reset back 3 days (combined with ISO8601 seems really easy to read/understand from the config definition itself) Something like that would definitely break consistency with the command line tool argument "by_duration", but if it seems clearer we should consider the tradeoff and not penalize the consumer API/configs. Thanks! Lianet On Sat, Nov 9, 2024 at 2:47 AM Ismael Juma <m...@ismaeljuma.com> wrote: > Thanks for the KIP, this is useful. A comment below. > > On Thu, Nov 7, 2024 at 4:51 PM Matthias J. Sax <mj...@apache.org> 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 configs seems to be > > desirable? -- It seems this might be a point of contention, and if the > > majority of people wants this new config so be it. I just wanted to > > stress my concerns about it. > > > I agree that we don't need a new config. We can simply use a value for the > existing config. I think a prefix followed by the relevant ISO8601 string > would be clear enough. For example, "by-duration:P23DT23H" or something > along those lines. I do find the "by-duration" description a bit confusing > for what we're doing here (i.e. current time - duration) although there is > precedent in the reset offsets tool. > > Ismael >