That's fair so I leave it up to you, Luke.

Cheers,
David

On Fri, May 12, 2023 at 10:58 AM Luke Chen <show...@gmail.com> wrote:

> Hi David,
>
> Thanks for the response.
> But I don't think the LEO-based leader election only benefit for this case.
> Like in unclean clear election case, we now randomly chose a out-of-sync
> replica to become the leader.
> This LEO-based leader election will help this case, too.
> Besides, not all producers use `acks=all`, thus, when using `acks=0` or
> `acks=1`, they can also benefit from LEO-based leader election.
>
> That's why I think this could be in a separate KIP, and after that's
> introduced, this KIP will be a further improvement based on that.
> Does that make sense?
>
> Actually, I don't strongly insist in doing this, if you still think they
> should be proposed together, I can update the KIP, too.
>
> Thank you.
> Luke
>
> On Fri, May 12, 2023 at 4:48 PM David Jacot <dja...@confluent.io.invalid>
> wrote:
>
> > Hi Luke,
> >
> > I disagree with this because we don't need the leader election change on
> > its own if we don't do this KIP. They have to go together or not at all
> in
> > my opinion. We need a KIP which designs the entire solution.
> >
> > Best,
> > David
> >
> > On Fri, May 12, 2023 at 10:33 AM Luke Chen <show...@gmail.com> wrote:
> >
> > > Hi Alexandre,
> > >
> > > Thanks for the thoughts.
> > > I've thought about it, and think I would choose to have a new leader
> > > election method to fix the problem we encountered, not this
> "backup-only"
> > > replica solution.
> > > But this is still an interesting idea. Like what you've said, this
> > solution
> > > can bring many benefits.
> > > So maybe you can create a proposal for it?
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Fri, May 12, 2023 at 4:21 PM Luke Chen <show...@gmail.com> wrote:
> > >
> > > > Hi Haruki,
> > > >
> > > > Yes, this scenario could happen.
> > > > I'm thinking we can fix it in step 6, when controller tried to get
> LEO
> > > > from B,C replicas, the B,C replica should stop fetcher for this
> > partition
> > > > immediately, before returning the LEO.
> > > > About if we need quorum-based or not, We can discuss in another KIP.
> > I'm
> > > > still thinking about it.
> > > >
> > > > Thank you.
> > > > Luke
> > > >
> > > >
> > > > On Fri, May 12, 2023 at 3:59 PM Luke Chen <show...@gmail.com> wrote:
> > > >
> > > >> Hi David,
> > > >>
> > > >> > It can't be in another KIP as it is required for your proposal to
> > > work.
> > > >> This is also an important part to discuss as it requires the
> > controller
> > > to
> > > >> do more operations on leader changes.
> > > >>
> > > >> Yes, I know this is a requirement for this KIP to work, and need a
> lot
> > > of
> > > >> discussion.
> > > >> So that's why I think it'd be better to have a separate KIP to write
> > the
> > > >> content and discussion.
> > > >> I've put the status of this KIP as "pending" and added a note on the
> > top
> > > >> of this KIP:
> > > >>
> > > >> Note: This KIP requires leader election change, which will be
> proposed
> > > in
> > > >> another KIP.
> > > >>
> > > >> Thanks.
> > > >> Luke
> > > >>
> > > >> On Thu, May 11, 2023 at 11:43 PM Alexandre Dupriez <
> > > >> alexandre.dupr...@gmail.com> wrote:
> > > >>
> > > >>> Hi, Luke,
> > > >>>
> > > >>> Thanks for your reply.
> > > >>>
> > > >>> 102. Whether such a replica could become a leader depends on what
> the
> > > >>> end-user wants to use it for and what tradeoffs they wish to make
> > down
> > > >>> the line.
> > > >>>
> > > >>> There are cases, for instance with heterogeneous or interregional
> > > >>> networks, where the difference in latency between subsets of
> brokers
> > > >>> can be high enough for the "slow replicas" to have a detrimental
> > > >>> impact on the ISR traffic they take part in. This can justify
> > > >>> permanently segregating them from ISR traffic by design. And, an
> > > >>> end-user could still prefer to have these "slow replicas" versus
> > > >>> alternative approaches such as mirroring for the benefits they can
> > > >>> bring, for instance: a) they belong to the same cluster with no
> added
> > > >>> admin and ops, b) benefit from a direct, simpler replication path,
> c)
> > > >>> require less infrastructure than a mirrored solution, d) could
> become
> > > >>> unclean leaders for failovers under disaster scenarios such as a
> > > >>> regional service outages.
> > > >>>
> > > >>> Thanks,
> > > >>> Alexandre
> > > >>>
> > > >>> Le jeu. 11 mai 2023 à 14:57, Haruki Okada <ocadar...@gmail.com> a
> > > écrit
> > > >>> :
> > > >>> >
> > > >>> > Hi, Luke.
> > > >>> >
> > > >>> > Though this proposal definitely looks interesting, as others
> > pointed
> > > >>> out,
> > > >>> > the leader election implementation would be the hard part.
> > > >>> >
> > > >>> > And I think even LEO-based-election is not safe, which could
> cause
> > > >>> silent
> > > >>> > committed-data loss easily.
> > > >>> >
> > > >>> > Let's say we have replicas A,B,C and A is the leader initially,
> and
> > > >>> > min.insync.replicas = 2.
> > > >>> >
> > > >>> > - 1. Initial
> > > >>> >     * A(leo=0), B(leo=0), C(leo=0)
> > > >>> > - 2. Produce a message to A
> > > >>> >     * A(leo=1), B(leo=0), C(leo=0)
> > > >>> > - 3. Another producer produces a message to A (i.e. as the
> > different
> > > >>> batch)
> > > >>> >     * A(leo=2), B(leo=0), C(leo=0)
> > > >>> > - 4. C replicates the first batch. offset=1 is committed (by
> > > >>> > acks=min.insync.replicas)
> > > >>> >     * A(leo=2), B(leo=0), C(leo=1)
> > > >>> > - 5. A loses ZK session (or broker session timeout in KRaft)
> > > >>> > - 6. Controller (regardless ZK/KRaft) doesn't store LEO in
> itself,
> > so
> > > >>> it
> > > >>> > needs to interact with each replica. It detects C has the largest
> > LEO
> > > >>> and
> > > >>> > decided to elect C as the new leader
> > > >>> > - 7. Before leader-election is performed, B replicates offset=1,2
> > > from
> > > >>> A.
> > > >>> > offset=2 is committed
> > > >>> >     * This is possible because even if A lost ZK session, A could
> > > >>> handle
> > > >>> > fetch requests for a while.
> > > >>> > - 8. Controller elects C as the new leader. B truncates its
> offset.
> > > >>> > offset=2 is lost silently.
> > > >>> >
> > > >>> > I have a feeling that we need quorum-based data replication? as
> > Divij
> > > >>> > pointed out.
> > > >>> >
> > > >>> >
> > > >>> > 2023年5月11日(木) 22:33 David Jacot <dja...@confluent.io.invalid>:
> > > >>> >
> > > >>> > > Hi Luke,
> > > >>> > >
> > > >>> > > > Yes, on second thought, I think the new leader election is
> > > >>> required to
> > > >>> > > work
> > > >>> > > for this new acks option. I'll think about it and open another
> > KIP
> > > >>> for it.
> > > >>> > >
> > > >>> > > It can't be in another KIP as it is required for your proposal
> to
> > > >>> work.
> > > >>> > > This is also an important part to discuss as it requires the
> > > >>> controller to
> > > >>> > > do more operations on leader changes.
> > > >>> > >
> > > >>> > > Cheers,
> > > >>> > > David
> > > >>> > >
> > > >>> > > On Thu, May 11, 2023 at 2:44 PM Luke Chen <show...@gmail.com>
> > > wrote:
> > > >>> > >
> > > >>> > > > Hi Ismael,
> > > >>> > > > Yes, on second thought, I think the new leader election is
> > > >>> required to
> > > >>> > > work
> > > >>> > > > for this new acks option. I'll think about it and open
> another
> > > KIP
> > > >>> for
> > > >>> > > it.
> > > >>> > > >
> > > >>> > > > Hi Divij,
> > > >>> > > > Yes, I agree with all of them. I'll think about it and let
> you
> > > >>> know how
> > > >>> > > we
> > > >>> > > > can work together.
> > > >>> > > >
> > > >>> > > > Hi Alexandre,
> > > >>> > > > > 100. The KIP makes one statement which may be considered
> > > >>> critical:
> > > >>> > > > "Note that in acks=min.insync.replicas case, the slow
> follower
> > > >>> might
> > > >>> > > > be easier to become out of sync than acks=all.". Would you
> have
> > > >>> some
> > > >>> > > > data on that behaviour when using the new ack semantic? It
> > would
> > > be
> > > >>> > > > interesting to analyse and especially look at the percentage
> of
> > > >>> time
> > > >>> > > > the number of replicas in ISR is reduced to the configured
> > > >>> > > > min.insync.replicas.
> > > >>> > > >
> > > >>> > > > The comparison data would be interesting. I can have a test
> > when
> > > >>> > > available.
> > > >>> > > > But this KIP will be deprioritized because there should be a
> > > >>> > > pre-requisite
> > > >>> > > > KIP for it.
> > > >>> > > >
> > > >>> > > > > A (perhaps naive) hypothesis would be that the
> > > >>> > > > new ack semantic indeed provides better produce latency, but
> at
> > > the
> > > >>> > > > cost of precipitating the slowest replica(s) out of the ISR?
> > > >>> > > >
> > > >>> > > > Yes, it could be.
> > > >>> > > >
> > > >>> > > > > 101. I understand the impact on produce latency, but I am
> not
> > > >>> sure
> > > >>> > > > about the impact on durability. Is your durability model
> built
> > > >>> against
> > > >>> > > > the replication factor or the number of min insync replicas?
> > > >>> > > >
> > > >>> > > > Yes, and also the new LEO-based leader election (not proposed
> > > yet).
> > > >>> > > >
> > > >>> > > > > 102. Could a new type of replica which would not be allowed
> > to
> > > >>> enter
> > > >>> > > > the ISR be an alternative? Such replica could attempt
> > replication
> > > >>> on a
> > > >>> > > > best-effort basis and would provide the permanent guarantee
> not
> > > to
> > > >>> > > > interfere with foreground traffic.
> > > >>> > > >
> > > >>> > > > You mean a backup replica, which will never become leader
> > > >>> (in-sync),
> > > >>> > > right?
> > > >>> > > > That's an interesting thought and might be able to become a
> > > >>> workaround
> > > >>> > > with
> > > >>> > > > the existing leader election. Let me think about it.
> > > >>> > > >
> > > >>> > > > Hi qiangLiu,
> > > >>> > > >
> > > >>> > > > > It's a good point that add this config and get better P99
> > > >>> latency, but
> > > >>> > > is
> > > >>> > > > this changing the meaning of "in sync replicas"? consider a
> > > >>> situation
> > > >>> > > with
> > > >>> > > > "replica=3 acks=2", when two broker fail and left only the
> > broker
> > > >>> that
> > > >>> > > > does't have the message, it is in sync, so will be elected as
> > > >>> leader,
> > > >>> > > will
> > > >>> > > > it cause a NOT NOTICED lost of acked messages?
> > > >>> > > >
> > > >>> > > > Yes, it will, so the `min.insync.replicas` config in the
> > > >>> broker/topic
> > > >>> > > level
> > > >>> > > > should be set correctly. In your example, it should be set to
> > 2,
> > > >>> so that
> > > >>> > > > when 2 replicas down, no new message write will be
> successful.
> > > >>> > > >
> > > >>> > > >
> > > >>> > > > Thank you.
> > > >>> > > > Luke
> > > >>> > > >
> > > >>> > > >
> > > >>> > > > On Thu, May 11, 2023 at 12:16 PM 67 <6...@gd67.com> wrote:
> > > >>> > > >
> > > >>> > > > > Hi Luke,
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > > > It's a good point that add this config and get better P99
> > > >>> latency, but
> > > >>> > > is
> > > >>> > > > > this changing the meaning of "in sync replicas"? consider a
> > > >>> situation
> > > >>> > > > with
> > > >>> > > > > "replica=3 acks=2", when two broker fail and left only the
> > > >>> broker that
> > > >>> > > > > does't have the message, it is in sync, so will be elected
> as
> > > >>> leader,
> > > >>> > > > will
> > > >>> > > > > it cause a *NOT NOTICED* lost of acked messages?
> > > >>> > > > >
> > > >>> > > > > qiangLiu
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > > > 在2023年05月10 12时58分,"Ismael Juma"<ism...@juma.me.uk>写道:
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > > > Hi Luke,
> > > >>> > > > >
> > > >>> > > > > As discussed in the other KIP, there are some subtleties
> when
> > > it
> > > >>> comes
> > > >>> > > to
> > > >>> > > > > the semantics of the system if we don't wait for all
> members
> > of
> > > >>> the isr
> > > >>> > > > > before we ack. I don't understand why you say the leader
> > > election
> > > >>> > > > question
> > > >>> > > > > is out of scope - it seems to be a core aspect to me.
> > > >>> > > > >
> > > >>> > > > > Ismael
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > > > On Wed, May 10, 2023, 8:50 AM Luke Chen <show...@gmail.com
> >
> > > >>> wrote:
> > > >>> > > > >
> > > >>> > > > > > Hi Ismael,
> > > >>> > > > > >
> > > >>> > > > > > No, I didn't know about this similar KIP! I hope I've
> known
> > > >>> that so
> > > >>> > > > that I
> > > >>> > > > > > don't need to spend time to write it again! :(
> > > >>> > > > > > I checked the KIP and all the discussions (here
> > > >>> > > > > > <
> > > >>>
> https://lists.apache.org/list?dev@kafka.apache.org:gte=100d:KIP-250
> > > >>> > > >).
> > > >>> > > > I
> > > >>> > > > > > think the consensus is that adding a client config to
> > > >>> `acks=quorum`
> > > >>> > > is
> > > >>> > > > > > fine.
> > > >>> > > > > > This comment
> > > >>> > > > > > <
> > > >>> https://lists.apache.org/thread/p77pym5sxpn91r8j364kmmf3qp5g65rn>
> > > >>> > > > from
> > > >>> > > > > > Guozhang pretty much concluded what I'm trying to do.
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > > *1. Add one more value to client-side acks config:   0:
> no
> > > acks
> > > >>> > > needed
> > > >>> > > > at
> > > >>> > > > > > all.   1: ack from the leader.   all: ack from ALL the
> ISR
> > > >>> replicas
> > > >>> > > > > >  quorum: this is the new value, it requires ack from
> enough
> > > >>> number of
> > > >>> > > > ISR
> > > >>> > > > > > replicas no smaller than majority of the replicas AND no
> > > >>> smaller
> > > >>> > > > > > than{min.isr}.2. Clarify in the docs that if a user wants
> > to
> > > >>> > > tolerate X
> > > >>> > > > > > failures, she needs to set client acks=all or acks=quorum
> > > >>> (better
> > > >>> > > tail
> > > >>> > > > > > latency than "all") with broker {min.sir} to be X+1;
> > however,
> > > >>> "all"
> > > >>> > > is
> > > >>> > > > not
> > > >>> > > > > > necessarily stronger than "quorum".*
> > > >>> > > > > >
> > > >>> > > > > > Concerns from KIP-250 are:
> > > >>> > > > > > 1. Introducing a new leader LEO based election method.
> This
> > > is
> > > >>> not
> > > >>> > > > clear in
> > > >>> > > > > > the KIP-250 and needs more discussion
> > > >>> > > > > > 2. The KIP-250 also tried to optimize the consumer
> latency
> > to
> > > >>> read
> > > >>> > > > messages
> > > >>> > > > > > beyond high watermark, which also has some discussion
> about
> > > >>> how to
> > > >>> > > > achieve
> > > >>> > > > > > that, and no conclusion
> > > >>> > > > > >
> > > >>> > > > > > Both of the above 2 concerns are out of the scope of my
> > > >>> current KIP.
> > > >>> > > > > > So, I think it's good to provide this `acks=quorum` or
> > > >>> > > > > > `acks=min.insync.replicas` option to users to give them
> > > another
> > > >>> > > choice.
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > > Thank you.
> > > >>> > > > > > Luke
> > > >>> > > > > >
> > > >>> > > > > >
> > > >>> > > > > > On Wed, May 10, 2023 at 8:54 AM Ismael Juma <
> > > ism...@juma.me.uk
> > > >>> >
> > > >>> > > wrote:
> > > >>> > > > > >
> > > >>> > > > > > > Hi Luke,
> > > >>> > > > > > >
> > > >>> > > > > > > Are you aware of
> > > >>> > > > > > >
> > > >>> > > > > > >
> > > >>> > > > > >
> > > >>> > > >
> > > >>> > >
> > > >>>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-250+Add+Support+for+Quorum-based+Producer+Acknowledgment
> > > >>> > > > > > > ?
> > > >>> > > > > > >
> > > >>> > > > > > > Ismael
> > > >>> > > > > > >
> > > >>> > > > > > > On Tue, May 9, 2023 at 10:14 PM Luke Chen <
> > > show...@gmail.com
> > > >>> >
> > > >>> > > wrote:
> > > >>> > > > > > >
> > > >>> > > > > > > > Hi all,
> > > >>> > > > > > > >
> > > >>> > > > > > > > I'd like to start a discussion for the KIP-926:
> > > introducing
> > > >>> > > > > > > > acks=min.insync.replicas config. This KIP is to
> > introduce
> > > >>> > > > > > > > `acks=min.insync.replicas` config value in producer,
> to
> > > >>> improve
> > > >>> > > the
> > > >>> > > > > > write
> > > >>> > > > > > > > throughput and still guarantee high durability.
> > > >>> > > > > > > >
> > > >>> > > > > > > > Please check the link for more detail:
> > > >>> > > > > > > >
> > > >>> > > > > > > >
> > > >>> > > > > > >
> > > >>> > > > > >
> > > >>> > > >
> > > >>> > >
> > > >>>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-926%3A+introducing+acks%3Dmin.insync.replicas+config
> > > >>> > > > > > > >
> > > >>> > > > > > > > Any feedback is welcome.
> > > >>> > > > > > > >
> > > >>> > > > > > > > Thank you.
> > > >>> > > > > > > > Luke
> > > >>> > > > > > > >
> > > >>> > > > > > >
> > > >>> > > > > >
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > > >
> > > >>> > > >
> > > >>> > >
> > > >>> >
> > > >>> >
> > > >>> > --
> > > >>> > ========================
> > > >>> > Okada Haruki
> > > >>> > ocadar...@gmail.com
> > > >>> > ========================
> > > >>>
> > > >>
> > >
> >
>

Reply via email to