Hi Kafka devs,

I opened a new JIRA to discuss acks=1 durability during planned leader 
transitions:

KAFKA-20554: https://issues.apache.org/jira/browse/KAFKA-20554

I reproduced a case on Kafka 4.1.2 where records acknowledged to an acks=1 
producer are later not readable from the topic after planned leader 
transitions. The same harness with acks=all produced no missing acknowledged 
records.

The cases I reproduced are:

1. Direct reassignment that removes the current leader

   [1,2] -> [3,2]

2. Replica reorder followed by preferred-leader election, with no replica-set 
change

   [1,2] -> [2,1]

3. Multi-step reassignment using preferred-leader elections

   [1,2] -> [2,1] -> [2,3] -> [3,2]

4. Graceful shutdown of the leader broker

In each case, the producer received successful acknowledgements, but some 
acknowledged records were not present when consuming the topic from the 
beginning afterwards.

My current understanding is that this is separate from KAFKA-19148 / 
KAFKA-19212. The new leader is not necessarily unclean in Kafka's ISR sense. 
Rather, with acks=1 the leader may acknowledge records after writing them only 
to its local log. If those records have not reached the high watermark before 
leadership moves, a clean leader election can still choose an ISR replica that 
does not contain that local suffix. The demoted broker then follows the new 
leader epoch and truncates the divergent tail.

So this appears to be allowed by the existing acks=1 durability contract, but 
it is operationally surprising because the loss can happen during planned 
administrative operations, not only after a hard leader failure.

I would like to get feedback on how the project wants to treat this:

- Is this expected acks=1 behavior that should be documented more explicitly?
- Should the documentation call out planned leader transitions specifically, 
such as reassignment, preferred-leader election, and graceful/controlled 
shutdown?
- Or does the project consider any of these planned operations to have stronger 
expectations than ordinary leader failure for acks=1 acknowledged records?

My initial suggestion is to treat this as a documentation clarification unless 
others think there is a behavioral issue here. In particular, I think it would 
help to make clear that unclean.leader.election.enable=false does not protect 
acks=1 acknowledgements, and that min.insync.replicas only affects producer 
success semantics when the producer uses acks=all.

If the consensus is that this is expected behavior but underdocumented, I can 
prepare a docs PR linked to the JIRA.

Thanks,
Julian
________________________________
Ultra Tendency International GmbH - Amtsgericht Stendal: HRB 26409 - 
Gesch?ftsf?hrer/CEO: Dr. Robert Neumann
August-Bebel-Str. 46, 39326 Colbitz, Germany - https://ultratendency.com - 
[email protected]

Reply via email to