Currently, with holding the assigned replicas(AR) for all partitions, controller is now able to elect new leaders by selecting the first replica of AR which occurs in both live replica set and ISR. If switching to the LEO-based strategy, controller context might need to be enriched or augmented to store those values. If retrieving those LEOs real-time, several rounds of RPCs are unavoidable which seems to violate the original intention of this KIP.
________________________________ 发件人: Dong Lin <lindon...@gmail.com> 发送时间: 2018年1月24日 14:31 收件人: dev@kafka.apache.org 主题: Re: KIP-250 Add Support for Quorum-based Producer Acknowledgment Hey Litao, Thanks for the KIP. I have one quick comment before you provide more detail on how to select the leader with the largest LEO. Do you think it would make sense to change the default behavior of acks=-1, such that broker will acknowledge the message once the message has been replicated to min.insync.replicas brokers? This would allow us to keep the same durability guarantee, improve produce request latency without having a new config. Thanks, Dong On Tue, Jan 23, 2018 at 8:38 PM, Litao Deng <denglitaoch...@gmail.com> wrote: > Hey folks. I would like to add a feature to support the quorum-based > acknowledgment for the producer request. We have been running a > modified version of Kafka on our testing cluster for weeks, the > improvement of P999 is significant with very stable latency. > Additionally, I have a proposal to achieve a similar data durability > as with the insync.replicas-based acknowledgment through LEO-based > leader election. > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 250+Add+Support+for+Quorum-based+Producer+Acknowledge >