This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 90087e416b0 KAFKA-19776: Fix log values for leader epoch (#20676)
90087e416b0 is described below

commit 90087e416b051433029dc33326c5e744c5a9710d
Author: Gaurav Narula <[email protected]>
AuthorDate: Fri Oct 10 18:14:56 2025 +0100

    KAFKA-19776: Fix log values for leader epoch (#20676)
    
    We missed a branch in #20671.
    
    This PR handles the else branch where we log about skipping the follower
    state change.
    
    Also updated the doc for the method as it was out of date.
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 core/src/main/scala/kafka/cluster/Partition.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 53228873e5d..b79668c9f9b 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -835,8 +835,7 @@ class Partition(val topicPartition: TopicPartition,
 
   /**
    * Make the local replica the follower by setting the new leader and ISR to 
empty
-   * If the leader replica id does not change and the new epoch is equal or one
-   * greater (that is, no updates have been missed), return false to indicate 
to the
+   * If the new leader epoch is less than current, return false to indicate to 
the
    * replica manager that state is already correct and the become-follower 
steps can
    * be skipped.
    */
@@ -882,7 +881,7 @@ class Partition(val topicPartition: TopicPartition,
           s"Previous leader $prevLeaderReplicaIdOpt and previous leader epoch 
was $prevLeaderEpoch.")
       } else {
         stateChangeLogger.info(s"Skipped the become-follower state change for 
$topicPartition with topic id $topicId, " +
-          s"partition registration $partitionRegistration and isNew=$isNew 
since it is already a follower with leader epoch $leaderEpoch.")
+          s"partition registration $partitionRegistration and isNew=$isNew 
since it is already a follower with leader epoch $prevLeaderEpoch.")
       }
 
       // We must restart the fetchers when the leader epoch changed regardless 
of

Reply via email to