bozhao12 commented on a change in pull request #11965:
URL: https://github.com/apache/kafka/pull/11965#discussion_r838290447



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -1300,6 +1300,62 @@ class ReplicaManagerTest {
     TestUtils.assertNoNonDaemonThreads(this.getClass.getName)
   }
 
+  @Test
+  def testFollowerShouldNotExecutePreferLeaderSelect(): Unit = {
+    val replicaManager = setupReplicaManagerWithMockedPurgatories(new 
MockTimer(time), aliveBrokerIds = Seq(0, 1, 2),
+      propsModifier = props => props.put(KafkaConfig.ReplicaSelectorClassProp, 
"org.apache.kafka.common.replica.RackAwareReplicaSelector"))
+    try {
+      val leaderBrokerId = 0
+      val follower1BrokerId = 1
+      val follower2BrokerId = 2
+      val brokerList = Seq[Integer](leaderBrokerId, follower1BrokerId, 
follower2BrokerId).asJava
+      val topicId = Uuid.randomUuid()
+      val tp0 = new TopicPartition(topic, 0)
+      val tidp0 = new TopicIdPartition(topicId, tp0)
+      initializeLogAndTopicId(replicaManager, tp0, topicId)
+      when(replicaManager.metadataCache.getPartitionReplicaEndpoints(
+        tp0,
+        new ListenerName("default")
+      )).thenReturn(Map(
+        leaderBrokerId -> new Node(leaderBrokerId, "host1", 9092, "rack-a"),
+        follower1BrokerId -> new Node(follower1BrokerId, "host2", 9092, 
"rack-a"),
+        follower2BrokerId -> new Node(follower2BrokerId, "host3", 9092, 
"rack-b")
+      ).toMap)
+
+      // Make this replica the follower
+      val leaderAndIsrRequest = new 
LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, 
brokerEpoch,
+        Seq(new LeaderAndIsrPartitionState()
+          .setTopicName(topic)
+          .setPartitionIndex(0)
+          .setControllerEpoch(0)
+          .setLeader(2)
+          .setLeaderEpoch(1)
+          .setIsr(brokerList)
+          .setZkVersion(0)
+          .setReplicas(brokerList)
+          .setIsNew(false)).asJava,
+        Collections.singletonMap(topic, topicId),
+        Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, 
"host3", 1)).asJava).build()
+      replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest, (_, _) => 
())
+      // Avoid the replica selector ignore the follower replica if it not have 
the data that need to fetch
+      
replicaManager.getPartitionOrException(tp0).updateFollowerFetchState(follower1BrokerId,
+        new LogOffsetMetadata(0), 0, 0, 0)

Review comment:
       @dajac If this line is removed, the test still passes, because the 
preferred read replica selected by the follower broker is the leader node, and 
`findPreferredReadReplica() `method will eventually return None.
   I want to prove through this case that if the change in 
`findPreferredReadReplica `is removed, in the current situation, follower will 
also execute the preferred read replica selection and can success select the 
replica.




-- 
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