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
>
>
>
>
> ------------------&nbsp;原始邮件&nbsp;------------------
> 发件人:
>                                                   "dev"
>                                                                 <
> ism...@juma.me.uk&gt;;
> 发送时间:&nbsp;2023年5月10日(星期三) 中午1:28
> 收件人:&nbsp;"dev"<dev@kafka.apache.org&gt;;
>
> 主题:&nbsp;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&gt; wrote:
>
> &gt; Hi Ismael,
> &gt;
> &gt; No, I didn't know about this similar KIP! I hope I've known that so
> that I
> &gt; don't need to spend time to write it again! :(
> &gt; I checked the KIP and all the discussions (here
> &gt; <
> https://lists.apache.org/list?dev@kafka.apache.org:gte=100d:KIP-250&gt;).
> I
> &gt; think the consensus is that adding a client config to `acks=quorum` is
> &gt; fine.
> &gt; This comment
> &gt; <https://lists.apache.org/thread/p77pym5sxpn91r8j364kmmf3qp5g65rn&gt;
> from
> &gt; Guozhang pretty much concluded what I'm trying to do.
> &gt;
> &gt;
> &gt;
> &gt;
> &gt;
> &gt;
> &gt;
> &gt;
> &gt; *1. Add one more value to client-side acks config:&nbsp;&nbsp; 0: no
> acks needed at
> &gt; all.&nbsp;&nbsp; 1: ack from the leader.&nbsp;&nbsp; all: ack from
> ALL the ISR replicas
> &gt;&nbsp; quorum: this is the new value, it requires ack from enough
> number of ISR
> &gt; replicas no smaller than majority of the replicas AND no smaller
> &gt; than{min.isr}.2. Clarify in the docs that if a user wants to tolerate
> X
> &gt; failures, she needs to set client acks=all or acks=quorum (better tail
> &gt; latency than "all") with broker {min.sir} to be X+1; however, "all"
> is not
> &gt; necessarily stronger than "quorum".*
> &gt;
> &gt; Concerns from KIP-250 are:
> &gt; 1. Introducing a new leader LEO based election method. This is not
> clear in
> &gt; the KIP-250 and needs more discussion
> &gt; 2. The KIP-250 also tried to optimize the consumer latency to read
> messages
> &gt; beyond high watermark, which also has some discussion about how to
> achieve
> &gt; that, and no conclusion
> &gt;
> &gt; Both of the above 2 concerns are out of the scope of my current KIP.
> &gt; So, I think it's good to provide this `acks=quorum` or
> &gt; `acks=min.insync.replicas` option to users to give them another
> choice.
> &gt;
> &gt;
> &gt; Thank you.
> &gt; Luke
> &gt;
> &gt;
> &gt; On Wed, May 10, 2023 at 8:54 AM Ismael Juma <ism...@juma.me.uk&gt;
> wrote:
> &gt;
> &gt; &gt; Hi Luke,
> &gt; &gt;
> &gt; &gt; Are you aware of
> &gt; &gt;
> &gt; &gt;
> &gt;
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-250+Add+Support+for+Quorum-based+Producer+Acknowledgment
> &gt
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-250+Add+Support+for+Quorum-based+Producer+Acknowledgment&gt>;
> &gt; ?
> &gt; &gt;
> &gt; &gt; Ismael
> &gt; &gt;
> &gt; &gt; On Tue, May 9, 2023 at 10:14 PM Luke Chen <show...@gmail.com&gt;
> wrote:
> &gt; &gt;
> &gt; &gt; &gt; Hi all,
> &gt; &gt; &gt;
> &gt; &gt; &gt; I'd like to start a discussion for the KIP-926: introducing
> &gt; &gt; &gt; acks=min.insync.replicas config. This KIP is to introduce
> &gt; &gt; &gt; `acks=min.insync.replicas` config value in producer, to
> improve the
> &gt; write
> &gt; &gt; &gt; throughput and still guarantee high durability.
> &gt; &gt; &gt;
> &gt; &gt; &gt; Please check the link for more detail:
> &gt; &gt; &gt;
> &gt; &gt; &gt;
> &gt; &gt;
> &gt;
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-926%3A+introducing+acks%3Dmin.insync.replicas+config
> &gt
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-926%3A+introducing+acks%3Dmin.insync.replicas+config&gt>;
> &gt; &gt;
> &gt; &gt; &gt; Any feedback is welcome.
> &gt; &gt; &gt;
> &gt; &gt; &gt; Thank you.
> &gt; &gt; &gt; Luke
> &gt; &gt; &gt;
> &gt; &gt;
> &gt;

Reply via email to