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
> >
>

Reply via email to