Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-17 Thread Luke Chen
; > > qiangLiu > > > > > > > > > > --原始邮件-- > > 发件人: > > "dev" > > < > > ism...@juma.me.uk; &g

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-16 Thread Kamal Chandraprakash
< > ism...@juma.me.uk; > 发送时间:2023年5月10日(星期三) 中午1:28 > 收件人:"dev" > 主题:Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config > > > > Hi Luke, > > As discussed in the other KIP, there

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-12 Thread David Jacot
That's fair so I leave it up to you, Luke. Cheers, David On Fri, May 12, 2023 at 10:58 AM Luke Chen 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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-12 Thread Luke Chen
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-12 Thread David Jacot
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 wrote: > Hi

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-12 Thread Luke Chen
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-12 Thread Luke Chen
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.

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-12 Thread Luke Chen
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-11 Thread Alexandre Dupriez
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-11 Thread Haruki Okada
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-11 Thread David Jacot
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-11 Thread Luke Chen
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-10 Thread 67
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-10 Thread Alexandre Dupriez
Hi, Luke, Thanks for the KIP. It clearly highlights the tradeoff between latency and durability and proposes an approach relaxing a durability constraint to provide lower ingestion latency. Please find a few comments/questions. 100. The KIP makes one statement which may be considered critical:

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-10 Thread Divij Vaidya
Thank you Luke for starting off this discussion. I have been thinking about this and other similar changes to the replication for a while now. The KIP that Ismael surfaced (where was that discussion thread hiding all this time!) addresses exactly the improvements that I have been wondering about.

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-09 Thread Ismael Juma
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

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-09 Thread Luke Chen
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 ). I think the consensus is that adding a

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-09 Thread Ismael Juma
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 wrote: > Hi all, > > I'd like to start a discussion for the KIP-926: introducing >

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-09 Thread Luke Chen
Hi Justine, Thanks for the comment. Yes, I know we have a configuration min.insync.replicas to avoid this case. And actually, that's my following paragraph is talking about. Please let me know if you have other comments. Thanks Luke Justine Olshan 於 2023年5月10日 週三 上午1:01 寫道: > Hey Luke, > > I

Re: [DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-09 Thread Justine Olshan
Hey Luke, I was taking a quick pass over the KIP and saw this line: >It looks perfect. But there's a caveat here. Like the doc said, acks=all will "wait for the *full set of in-sync replicas *to acknowledge the record", so if there's only 1 replica in in-sync replicas, it will have the same

[DISCUSS] KIP-926: introducing acks=min.insync.replicas config

2023-05-09 Thread Luke Chen
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: