Hello Deqi,

Thanks for bringing this KIP, and sorry for getting back to you so late.

I do think that separating the reset policy for the two scenarios: 1) when
we get an out-of-range when polling records, likely due to log being
truncated, 2) when we start fetching and there's no committed offset, would
be preferable as we have seen many places where people would prefer to use
different strategies for these two scenarios. At the same time, I also
share other's concerns that the current proposal is trying to mingle too
many features together which makes it unnecessarily complicated and also
makes the compatibility story trickier.

My readings are that, you want to achieve the following things within this
KIP:

a) separate the two scenarios for reset strategies, as I mentioned above.
This to me is the most compelling motivation.
b) introduce new reset policies, a.k.a."nearest"in addition to earliest and
latest. This has been proposed quite a while ago to add more flexibilities
in the policies.
c) tailor the reset policy of a topic for a specific consumer group. I.e.
when a consumer group starts consuming, we want to let it start from
"latest", but once the consumer group starts, newly added partitions would
be using "earliest" instead to avoid data loss.

I think trying to compound all these things in this one KIP makes it a bit
too mingled, and complicated. Plus, we also need to make sure that we are
compatible with the old behaviors if users only set "earliest" or "latest",
and expect that to impact both scenarios.

I think about them for a bit and here's my 2c: how about, we simplify this
KIP in the following way. The first three rows are existing strategies that
we do not change for compatibility.

--------------------------------------

*auto.offset.reset                    when out-of-range
 when no committed offsets found*
none                                         throw exception
         throw exception
earliest                                     reset to earliest
            reset to earliest
latest                                        reset to latest
               reset to latest

*auto.offset.reset.on.start*
earliest_on_start                      throw exception
    reset to earliest
latest_on_start                         throw exception
      reset to latest




















On Fri, Jul 8, 2022 at 7:01 AM hudeqi <16120...@bjtu.edu.cn> wrote:

> Regarding the option to integrate repair logic in "latest", I understand
> your concern about this approach: backward compatibility.
> But we should have a consensus: the problem of data loss due to expand
> partitions is indeed caused by kafka's own design mechanism. The user
> configuration "latest" may be due to the consideration of not wanting to
> consume from earliest when firstly deploy app, or too much lag, or
> consumption exceeds the maximum offset, and then consume directly from the
> latest. As for expanding partition, the user will definitely not want to
> consume from the latest, unless he clearly knows what this means.
> Therefore, it is necessary to solve this problem, at the same time, without
> causing other problems.
> Therefore, for the method of adding an "init.offset.reset" option, there
> will be a problem, that is, this configuration must be set to "earliest" to
> avoid this situation, but it will also cause the new group to be consumed
> from earliest. , which goes against the idea of ​​consuming from the latest
> at the beginning (brings other problems).
> The same is true for the method of setting auto.offset.reset to "earliest"
> and seekingToEnd on new deployments: in order to avoid this case,
> "auto.offset.reset" has no choice but to set "earliest", when the
> consumption is advanced, it will also reset to the earliest, causing
> duplication (bringing other problems).
> So I think it's best to fix it in a black box to fundamentally solve it.
> It does not require users to perceive this problem, nor does the user's
> understanding of "auto.offset.reset" need to be changed, and there will be
> no complexity caused by redundant parameter configuration(and users doesn't
> necessarily know how to combine these parameters to use it). As for the
> compatibility issue, I think it is enough to enrich the test cases after
> the repair, what do you think?
>
> &quot;Matthias J. Sax&quot; &lt;mj...@apache.org&gt;写道:
> > I am not sure if we can/should change the behavior of existing
> > "latest/earliest" due to backward compatibility concerns. While I agree
> > that many users might not know the fine details how both behave, it
> > would still be a change that might break other people that do understand
> > the details and rely on it.
> >
> > I also agree that having both "latest" and "safe_latest" might be
> > difficult, as users might not know which one to choose?
> >
> > Maybe we should have two configs instead of one? `auto.offset.reset`, as
> > the name suggests, resets the offset automatically, and thus it's
> > current behavior is actually well defined and sound. -- What seems to be
> > missing is an `init.offset` config that is only used if there is _no_
> > committed offsets, but it's not used when the consumer has a position
> > already (either via getting a committed offset or via seek())?
> >
> >
> > For the original use-case you mentioned, that you want to start from
> > "latest" when the app starts, but if a new partition is added you want
> > to start from "earliest" it seem that the right approach would be to
> > actually configure "earliest", and when the app is deployed for the
> > first time, use a `seekToEnd()` to avoid triggering auto-offset-reset?
> >
> >
> > Thoughts?
> >
> >
> > -Matthias
> >
> >
> > On 7/1/22 6:03 AM, hudeqi wrote:
> > > Thanks for your attention and reply.
> > > Having chatted with Guozhang Wang at KAFKA-12478 before, I came up
> with an idea similar to yours. It's just not implemented on the client
> side, but on the server side: Firstly, find out all the groups subscribed
> to this topic before extending partitions, and then let these groups commit
> an initial offset 0 for these new expanded partitions (also using
> adminClient). Finally, the real process of adding partitions is carried
> out. In this way, the problem can also be completely solved.
> > >
> > > Best,
> > > hudeqi
> > >
> > > &quot;Matthew Howlett&quot; &lt;m...@confluent.io.INVALID&gt;写道:
> > >> My first reaction also is that the proposed configuration is surely
> too
> > >> complicated.
> > >>
> > >> It seems like an ideal solution from a usability perspective (always
> a good
> > >> place to start) would be if the consumer just automatically behaved
> in this
> > >> way. To make that work:
> > >> 1. auto.offset.reset=latest would need to behave like
> > >> auto.offset.reset=earliest in the case where a consumer is in a
> group, and
> > >> is assigned a newly created partition. This might seem a bit too
> "magic",
> > >> but from the perspective of the group, I think it makes conceptual
> sense
> > >> and people wouldn't find it surprising. Also, I don't think anyone
> would be
> > >> relying on the old behavior.
> > >> 2. The group would need to detect the newly created partitions and
> > >> rebalance pretty quickly (this is not the case currently). The longer
> the
> > >> delay, the more tenuous the idea of changing the auto.offset.reset
> behavior
> > >> in this special circumstance.
> > >>
> > >> I have a feeling this approach has implementation challenges (haven't
> > >> thought deeply), just throwing it out there.
> > >>
> > >>
> > >> On Wed, Jun 29, 2022 at 4:57 AM David Jacot <da...@apache.org> wrote:
> > >>
> > >>> Thanks for the KIP.
> > >>>
> > >>> I read it and I am also worried by the complexity of the new
> > >>> configurations. They are not easy to grasp. I need to digest it a
> bit more,
> > >>> I think.
> > >>>
> > >>> Best,
> > >>> David
> > >>>
> > >>> Le mer. 29 juin 2022 à 02:25, Matthias J. Sax <mj...@apache.org> a
> écrit :
> > >>>
> > >>>> Thanks for the KIP.
> > >>>>
> > >>>> I don't think I fully digested the proposal yet, but my first
> reaction
> > >>>> is: this is quite complicated. Frankly, I am worried about
> complexity
> > >>>> and usability.
> > >>>>
> > >>>> Especially the option `safe_latest` is a "weird" one IMHO, and
> `nearest`
> > >>>> is even more complex.
> > >>>>
> > >>>> The problem at hand (as I understand it from the Jira) is a real
> one,
> > >>>> but I am wondering if it would be something that should be
> addressed by
> > >>>> the application? If you pass in strategy `none`, and a new
> partition is
> > >>>> added, you can react to it by custom code. For regular startup you
> can
> > >>>> still go with "latest" to avoid reprocessing the history.
> > >>>>
> > >>>> Adding "latest/earliest_on_start" seems useful, as it seems to also
> > >>>> address https://issues.apache.org/jira/browse/KAFKA-3370
> > >>>>
> > >>>>
> > >>>> -Matthias
> > >>>>
> > >>>>
> > >>>> On 6/7/22 12:55 AM, hudeqi wrote:
> > >>>>> I think so too, what about Guozhang Wang and Luke Chen? Can I
> initiate
> > >>> a
> > >>>> voting process?
> > >>>>>
> > >>>>> Best,
> > >>>>> hudeqi
> > >>>>>
> > >>>>> &gt; -----原始邮件-----
> > >>>>> &gt; 发件人: "邓子明" <dengzim...@growingio.com>
> > >>>>> &gt; 发送时间: 2022-06-07 10:23:37 (星期二)
> > >>>>> &gt; 收件人: dev@kafka.apache.org
> > >>>>> &gt; 抄送:
> > >>>>> &gt; 主题: Re: [DISCUSS] KIP-842: Add richer group offset reset
> > >>> mechanisms
> > >>>>> &gt;
> > >>>>> </dengzim...@growingio.com>
> > >>>>
> > >>>
>


-- 
-- Guozhang

Reply via email to