[
https://issues.apache.org/jira/browse/KAFKA-19148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18078681#comment-18078681
]
Julian Bergner edited comment on KAFKA-19148 at 5/6/26 1:58 PM:
----------------------------------------------------------------
Hi [~bafli],
I confirmed that with {{{}acks=1{}}}, Kafka can lose producer-acknowledged
records during planned leader transitions, not only during hard leader failure.
I reproduced this on Kafka 4.1.2 with the following cases:
* direct reassignment that removes the leader, for example {{[1,2] → [3,2]}}
* preferred-leader election after replica reorder, for example {{[1,2] →
[2,1]}}
* multi-step reassignment, for example {{[1,2] → [2,1] → [2,3] → [3,2]}}
* graceful shutdown of the leader
In all cases, the producer received successful acknowledgements, but some
acknowledged records were not later readable from the topic when consuming from
the beginning. With the same harness using {{{}acks=all{}}}, I saw no loss.
This appears to be separate from KAFKA-19148/KAFKA-19212. The new leader is not
necessarily unclean in Kafka’s ISR sense. The issue is that {{acks=1}} allows
the current leader to acknowledge records after writing them only to its local
log. If those records have not reached the high watermark, a later clean leader
election can still choose an ISR replica that does not contain that local
suffix. Once leadership moves, the demoted broker follows the new leader epoch
and truncates the divergent tail.
So the important distinction is: {{unclean.leader.election.enable=false}} does
not protect {{acks=1}} acknowledgements, and {{min.insync.replicas}} only
affects producer success semantics when the producer uses {{{}acks=all{}}}.
Since KAFKA-19212 fixed the unclean-election false-positive, I’ll close
KAFKA-19148 as resolved. I opened a separate JIRA KAFKA-20554 for the
{{acks=1}} behavior with the reproducer, logs, and observed cases, so the
project can discuss whether this should be handled as
expected-but-underdocumented behavior, a documentation clarification, or a
possible change to planned leader-transition semantics.
was (Author: JIRAUSER309281):
Hi [~bafli],
I confirmed that with {{{}acks=1{}}}, Kafka can lose producer-acknowledged
records during planned leader transitions, not only during hard leader failure.
I reproduced this on Kafka 4.1.2 with the following cases:
* direct reassignment that removes the leader, for example {{[1,2] → [3,2]}}
* preferred-leader election after replica reorder, for example {{[1,2] →
[2,1]}}
* multi-step reassignment, for example {{[1,2] → [2,1] → [2,3] → [3,2]}}
* graceful shutdown of the leader
In all cases, the producer received successful acknowledgements, but some
acknowledged records were not later readable from the topic when consuming from
the beginning. With the same harness using {{{}acks=all{}}}, I saw no loss.
This appears to be separate from KAFKA-19148/KAFKA-19212. The new leader is not
necessarily unclean in Kafka’s ISR sense. The issue is that {{acks=1}} allows
the current leader to acknowledge records after writing them only to its local
log. If those records have not reached the high watermark, a later clean leader
election can still choose an ISR replica that does not contain that local
suffix. Once leadership moves, the demoted broker follows the new leader epoch
and truncates the divergent tail.
So the important distinction is: {{unclean.leader.election.enable=false}} does
not protect {{acks=1}} acknowledgements, and {{min.insync.replicas}} only
affects producer success semantics when the producer uses {{{}acks=all{}}}.
Since KAFKA-19212 fixed the unclean-election false-positive, I’ll close
KAFKA-19148 as resolved. I’ll open a separate JIRA for the {{acks=1}} behavior
with the reproducer, logs, and observed cases, so the project can discuss
whether this should be handled as expected-but-underdocumented behavior, a
documentation clarification, or a possible change to planned leader-transition
semantics.
> Potential Unclean Leader Election in KRaft Despite
> unclean.leader.election.enable=false
> ---------------------------------------------------------------------------------------
>
> Key: KAFKA-19148
> URL: https://issues.apache.org/jira/browse/KAFKA-19148
> Project: Kafka
> Issue Type: Bug
> Components: controller
> Affects Versions: 3.9.0, 4.0.0
> Reporter: Julian Bergner
> Assignee: Julian Bergner
> Priority: Critical
> Fix For: 4.1.0
>
> Attachments: Readme_Kraft.md, Readme_Zookeeper.md,
> docker-compose_kraft.yml, docker-compose_zookeeper.yml
>
>
> *Issue Summary:*
> We're observing unclean leader election even though
> {{{}unclean.leader.election.enable=false{}}}.
> *Scenario:*
> During a partition reassignment, if we promote a non-ISR broker to leader and
> simultaneously remove the current leader from the ISR, Kafka still elects a
> new leader from outside the ISR. This behavior contradicts the expected
> behavior when unclean leader election is explicitly disabled.
> *Details:*
> * *Original ISR:* [1, 2]
> * *New ISR after reassignment:* [3, 2]
> *Kafka Versions Tested:*
> * Kafka 4.0.0 (KRaft mode)
> * Kafka 3.9.0 (Kraft mode)
> * Kafka 3.9.0 (Zookeeper mode)
> *Observation:*
> * The behaviour differs between the two modes.
> * In Kraft, unclean leader election occurred, which should not happen with
> the config set to {{{}false{}}}.
> * In Zookeeper no unclean leader election occurred.
> *Attachments:*
> Docker Compose files and reproduction steps for both:
> * Kafka 4.0.0 (KRaft)
> * Kafka 3.9.0 (Zookeeper)
--
This message was sent by Atlassian Jira
(v8.20.10#820010)