mumrah commented on PR #17118: URL: https://github.com/apache/kafka/pull/17118#issuecomment-2334401889
I could not reproduce this locally, but we do see this test failing occasionally during CI. Here's one such failure: https://ge.apache.org/s/a7oj4c6hibpn4/tests/task/:metadata:test/details/org.apache.kafka.controller.QuorumControllerTest/testBootstrapZkMigrationRecord()/1/output ``` [2024-08-24 13:31:16,023] INFO [QuorumController id=0] Creating new QuorumController with clusterId v1UcC5zIR1SUQujnl7u31w. (org.apache.kafka.controller.QuorumController:1909) [2024-08-24 13:31:16,024] INFO [LocalLogManager 0] Node 0: registered MetaLogListener 1234229248 (org.apache.kafka.metalog.LocalLogManager:681) [2024-08-24 13:31:16,031] DEBUG [LocalLogManager 0] Node 0: running log check. (org.apache.kafka.metalog.LocalLogManager:522) [2024-08-24 13:31:16,031] DEBUG [LocalLogManager 0] Node 0: Executing handleLeaderChange LeaderAndEpoch(leaderId=OptionalInt[0], epoch=1) (org.apache.kafka.metalog.LocalLogManager:564) [2024-08-24 13:31:16,031] DEBUG [QuorumController id=0] Executing handleLeaderChange[1]. (org.apache.kafka.controller.QuorumController:577) [2024-08-24 13:31:16,032] INFO [QuorumController id=0] In the new epoch 1, the leader is (none). (org.apache.kafka.controller.QuorumController:1175) [2024-08-24 13:31:16,032] DEBUG [QuorumController id=0] Processed handleLeaderChange[1] in 387 us (org.apache.kafka.controller.QuorumController:510) [2024-08-24 13:31:16,032] DEBUG [QuorumController id=0] Executing handleLeaderChange[1]. (org.apache.kafka.controller.QuorumController:577) [2024-08-24 13:31:16,032] INFO [QuorumController id=0] Becoming the active controller at epoch 1, next write offset 1. (org.apache.kafka.controller.QuorumController:1171) [2024-08-24 13:31:16,032] DEBUG [QuorumController id=0] Processed handleLeaderChange[1] in 361 us (org.apache.kafka.controller.QuorumController:510) [2024-08-24 13:31:16,033] WARN [QuorumController id=0] Performing controller activation. The metadata log appears to be empty. Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster. (org.apache.kafka.controller.QuorumController:108) [2024-08-24 13:31:16,036] DEBUG [QuorumController id=0] Got exception while running completeActivation[1](731556340). Invoking handleException. (org.apache.kafka.queue.KafkaEventQueue:137) org.apache.kafka.raft.errors.NotLeaderException: Append failed because the given epoch 0 is stale. Current leader epoch = 1 at org.apache.kafka.metalog.LocalLogManager$SharedLogData.tryAppend(LocalLogManager.java:246) at org.apache.kafka.metalog.LocalLogManager$SharedLogData.tryAppend(LocalLogManager.java:228) at org.apache.kafka.metalog.LocalLogManager.prepareAppend(LocalLogManager.java:736) at org.apache.kafka.controller.QuorumController$ControllerWriteEvent.lambda$run$0(QuorumController.java:821) at org.apache.kafka.controller.QuorumController.appendRecords(QuorumController.java:914) at org.apache.kafka.controller.QuorumController$ControllerWriteEvent.run(QuorumController.java:815) at org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:132) at org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:215) at org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:186) at java.base/java.lang.Thread.run(Thread.java:829) ``` We see that the event `completeActivation[1]` has the correct controller epoch of 1, but the log layer is saying we gave 0. Tracing this down, it looks like LocalLogManager is passing `leader.epoch()` instead of the given epoch here. If I add a short sleep to LocalLogManager in the LeaderChangeBatch code, I can reproduce this failure consistently ``` listenerData.handleLeaderChange(entryOffset, batch.newLeader); if (batch.newLeader.epoch() > leader.epoch()) { Thread.sleep(100); leader = batch.newLeader; } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
