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.

Let me state certain points here and tell me what you think about them.

#1 We need to change the leader election if we introduce the new ack=min.isr
I will expand on Ismael's comment about the necessity to change the leader
election with an example.
1. {A, B, C} are in ISR, A is leader, min.insync.replicas=2
2. Write comes in with acks=min.insync.replicas and {A,B} receives the
write and it gets acknowledged to the producer. C hasn't still received the
write.
3. A fails. Leadership transfers to C.
4. C hasn't received the write in step 2 and hence, it will ask B to
truncate itself and match the prefix of C.

As you can observe, if we don't change the leader election strategy to
choosing a leader with the largest LEO, we may end up in a situation where
we are losing ACK'ed messages. This is a durability loss.

#2 Now that we have established based on statement 1 above that it is
necessary to modify the leader election, I believe we definitely should do
it (and revive conversation from KIP-250). Determining the ISR with the
largest LEO comes with a cost of multiple round trips with controllers.
This is an acceptable cost because it improves the steady state scenario
(lower latency for writes) while adding additional overhead of
rare/exceptional scenarios (leadership failover).
Another advantage of choosing the leader with the largest LEO is evident in
case of an unclean leader election. We can extend this new leader election
logic to choose the out-of-sync replica with the largest LEO in case of
unclean leader election. This will reduce the amount of data loss in such a
scenario. I have a draft for this here
<https://docs.google.com/document/d/1iXIbXPNGy_Y2hLvb3SQSDSP8NGQZkWGLED1pT5ASLIg/edit?usp=sharing>
but
I never ended up writing a KIP for it.

#3 Now, if we agree that we need to change the leader election to
improve steady state, should we consider a raft-like quorum based algorithm
instead of the current one? IMO, yes we should implement a quorum based
algorithm, but not in the scope of this change. That is a bigger change and
requires a different KIP which shouldn't block the immediate advantages of
your proposal.

#4 Changes to the replication protocol are tricky and full of edge case
scenarios. How do we develop in the future and gain confidence about the
changes? This is where formal models like TLA+ comes into the picture.
Modeling Kafka's replication protocol in TLA+ helps us in demonstrating
provable semantics AND it also helps in quick iteration of ideas. As an
example, for your proposal, we can extend the (work in progress) model
here:
https://github.com/divijvaidya/kafka-specification/blob/master/Kip405.tla#L112
and assert that the invariants hold true even after we make the change
about ack (currently model doesn't support ack, it only supports changing
HW which will remain same even after this KIP).

Once we align on the above four points, I will be happy to work with you to
explore the possible solutions in detail.

--
Divij Vaidya



On Wed, May 10, 2023 at 6:59 AM Ismael Juma <ism...@juma.me.uk> wrote:

> 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