jsancio commented on code in PR #19854:
URL: https://github.com/apache/kafka/pull/19854#discussion_r2195283553


##########
raft/src/main/java/org/apache/kafka/raft/FollowerState.java:
##########
@@ -137,6 +137,11 @@ public void resetFetchTimeoutForSuccessfulFetch(long 
currentTimeMs) {
         hasFetchedFromLeader = true;
     }
 
+    public void resetFetchTimeoutAfterDiscoveringLeader(long currentTimeMs) {

Review Comment:
   Let's document these 3 methods: `resetFetchTimeoutForSuccessfulFetch`, 
`resetFetchTimeoutAfterDiscoveringLeader` and `overrideFetchTimeout`.
   
   How about removing code duplication with:
   ```java
       public void resetFetchTimeoutForSuccessfulFetch(long currentTimeMs) {
           resetFetchTimeout(currentTimeMs);
           hasFetchedFromLeader = true;
       }
   
       public void resetFetchTimeout(long currentTimeMs) {
          overrideFetchTimeout(currentTimeMs, fetchTimeoutMs);
       }
   ```



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -2569,13 +2569,15 @@ private void maybeTransition(
             } else {
                 transitionToUnattached(epoch, OptionalInt.empty());
             }
-        } else if (
-                leaderId.isPresent() &&
-                (!quorum.hasLeader() || leaderEndpoints.size() > 
quorum.leaderEndpoints().size())
-        ) {
-            // The request or response indicates the leader of the current 
epoch
-            // which are currently unknown or the replica has discovered more 
endpoints
-            transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, 
currentTimeMs);
+        } else if (leaderId.isPresent()) {
+            if (quorum.isFollowerObserver()) {

Review Comment:
   Why don't we also need to do this for voters? If so, should the logic always 
re-transition to follower?



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to