[ https://issues.apache.org/jira/browse/KAFKA-19148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17949579#comment-17949579 ]
Azhar Ahmed commented on KAFKA-19148: ------------------------------------- Ran the test for 3.9 Kraft cluster using the repro steps in the attached Readme_Kraft.md I was able to reproduce the issue and hence its a problem. {code:java} Processed 29000000 messages so far. Processed 30000000 messages so far. Processed 31000000 messages so far. Inconsistency found: expected 31451085 but got 31457229 Inconsistency found: expected 31458253 but got 31456205 Inconsistency found: expected 31457229 but got 31455181 Inconsistency found: expected 31456205 but got 31454157 Inconsistency found: expected 31455181 but got 31453133 Inconsistency found: expected 31454157 but got 31458253 Processed 32000000 messages so far. Processed 33000000 messages so far. Processed 34000000 messages so far. {code} Next step: Deep dive into code and identify problematic piece of logic > 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: Azhar Ahmed > Priority: Critical > 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)