Hi, Inline with the *backup-only* replica approach, KIP-491: Preferred Leader Deprioritized List <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=120736982> was proposed to mark a replica as the lowest priority to become leader.
On Sat, May 13, 2023 at 6:53 AM 67 <iamg...@qq.com.invalid> 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 > > > > > ------------------ 原始邮件 ------------------ > 发件人: > "dev" > < > ism...@juma.me.uk>; > 发送时间: 2023年5月10日(星期三) 中午1:28 > 收件人: "dev"<dev@kafka.apache.org>; > > 主题: Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config > > > > 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 > > > <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 > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-926%3A+introducing+acks%3Dmin.insync.replicas+config>>; > > > > > > > Any feedback is welcome. > > > > > > > > Thank you. > > > > Luke > > > > > > > > >