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 makes one statement which may be considered critical:
"Note that in acks=min.insync.replicas case, the slow follower might
be easier to become out of sync than acks=all.". Would you have some
data on that behaviour when using the new ack semantic? It would be
interesting to analyse and especially look at the percentage of time
the number of replicas in ISR is reduced to the configured
min.insync.replicas.

The comparison data would be interesting. I can have a test when available.
But this KIP will be deprioritized because there should be a pre-requisite
KIP for it.

> A (perhaps naive) hypothesis would be that the
new ack semantic indeed provides better produce latency, but at the
cost of precipitating the slowest replica(s) out of the ISR?

Yes, it could be.

> 101. I understand the impact on produce latency, but I am not sure
about the impact on durability. Is your durability model built against
the replication factor or the number of min insync replicas?

Yes, and also the new LEO-based leader election (not proposed yet).

> 102. Could a new type of replica which would not be allowed to enter
the ISR be an alternative? Such replica could attempt replication on a
best-effort basis and would provide the permanent guarantee not to
interfere with foreground traffic.

You mean a backup replica, which will never become leader (in-sync), right?
That's an interesting thought and might be able to become a workaround with
the existing leader election. Let me think about it.

Hi qiangLiu,

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

Yes, it will, so the `min.insync.replicas` config in the broker/topic level
should be set correctly. In your example, it should be set to 2, so that
when 2 replicas down, no new message write will be successful.


Thank you.
Luke


On Thu, May 11, 2023 at 12:16 PM 67 <6...@gd67.com> 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
>
>
> 在2023年05月10 12时58分,"Ismael Juma"<ism...@juma.me.uk>写道:
>
>
> 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
> > > ?
> > >
> > > 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