frankvicky commented on code in PR #19924:
URL: https://github.com/apache/kafka/pull/19924#discussion_r2134981124


##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -474,20 +475,21 @@ class ReplicaManagerTest {
       }
 
       // Make this replica the follower
-      val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(0, 0, 
brokerEpoch,
-        Seq(new LeaderAndIsrRequest.PartitionState()
-          .setTopicName(topic)
-          .setPartitionIndex(0)
-          .setControllerEpoch(0)
-          .setLeader(1)
-          .setLeaderEpoch(1)
-          .setIsr(brokerList)
-          .setPartitionEpoch(0)
-          .setReplicas(brokerList)
-          .setIsNew(false)).asJava,
-        topicIds,
-        Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
-      rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ())
+      val delta1 = new TopicsDelta(TopicsImage.EMPTY)
+      delta1.replay(new 
TopicRecord().setName(topic).setTopicId(topicIds.get(topic)))
+      val record1 = new PartitionRecord()
+        .setPartitionId(0)
+        .setTopicId(topicIds.get(topic))
+        .setReplicas(brokerList)
+        .setIsr(brokerList)
+        .setRemovingReplicas(util.List.of())
+        .setAddingReplicas(util.List.of())
+        .setLeader(brokerList.get(1))
+        .setLeaderEpoch(1)
+        .setPartitionEpoch(0)
+      delta1.replay(record1)
+      val leaderMetadataImage1 = imageFromTopics(delta1.apply())

Review Comment:
   ditto



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -945,20 +947,9 @@ class ReplicaManagerTest {
         new 
LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints.asJava), None)
 
       // Make this replica the leader.
-      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, 
brokerEpoch,
-        Seq(new LeaderAndIsrRequest.PartitionState()
-          .setTopicName(topic)
-          .setPartitionIndex(0)
-          .setControllerEpoch(0)
-          .setLeader(0)
-          .setLeaderEpoch(0)
-          .setIsr(brokerList)
-          .setPartitionEpoch(0)
-          .setReplicas(brokerList)
-          .setIsNew(true)).asJava,
-        topicIds.asJava,
-        Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
-      replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => 
())
+      val delta = topicsCreateDelta(brokerList.get(0), isStartIdLeader = true, 
partitions = List(0), List.empty, topic, topicIds(topic))
+      val leaderMetadataImage = imageFromTopics(delta.apply())

Review Comment:
   ditto



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -2593,8 +2584,9 @@ class ReplicaManagerTest {
     val replicaManager = 
setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager,
 List(tp), config = config)
 
     try {
-      val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp.topic), 
tp, Seq(0, 1), new LeaderAndIsr(0, List(0, 1).map(Int.box).asJava))
-      replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => 
())
+      val delta = topicsCreateDelta(0, isStartIdLeader = true, partitions = 
List(0), List.empty, topic, topicIds(topic))
+      val leaderMetadataImage = imageFromTopics(delta.apply())

Review Comment:
   ditto



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -4180,7 +4156,21 @@ class ReplicaManagerTest {
       assertEquals(0, 
brokerTopicStats.allTopicsStats.buildRemoteLogAuxStateRequestRate.count)
       assertEquals(0, 
brokerTopicStats.allTopicsStats.failedBuildRemoteLogAuxStateRate.count)
 
-      replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => 
())
+      val delta = new TopicsDelta(TopicsImage.EMPTY)
+      delta.replay(new 
TopicRecord().setName(topic).setTopicId(topicIds.get(topic)))
+      val record = new PartitionRecord()
+        .setPartitionId(0)
+        .setTopicId(topicIds.get(topic))
+        .setReplicas(util.List.of(0, 1))
+        .setIsr(util.List.of(0, 1))
+        .setRemovingReplicas(util.List.of())
+        .setAddingReplicas(util.List.of())
+        .setLeader(1)
+        .setLeaderEpoch(0)
+        .setPartitionEpoch(0)
+      delta.replay(record)
+      val leaderMetadataImage = imageFromTopics(delta.apply())

Review Comment:
   ditto



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -451,20 +451,21 @@ class ReplicaManagerTest {
       partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
         new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None)
       // Make this replica the leader.
-      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, 
brokerEpoch,
-        Seq(new LeaderAndIsrRequest.PartitionState()
-          .setTopicName(topic)
-          .setPartitionIndex(0)
-          .setControllerEpoch(0)
-          .setLeader(0)
-          .setLeaderEpoch(0)
-          .setIsr(brokerList)
-          .setPartitionEpoch(0)
-          .setReplicas(brokerList)
-          .setIsNew(false)).asJava,
-        topicIds,
-        Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
-      rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
+      val delta = new TopicsDelta(TopicsImage.EMPTY)
+      delta.replay(new 
TopicRecord().setName(topic).setTopicId(topicIds.get(topic)))
+      val record = new PartitionRecord()
+        .setPartitionId(0)
+        .setTopicId(topicIds.get(topic))
+        .setReplicas(brokerList)
+        .setIsr(brokerList)
+        .setRemovingReplicas(util.List.of())
+        .setAddingReplicas(util.List.of())
+        .setLeader(brokerList.get(0))
+        .setLeaderEpoch(0)
+        .setPartitionEpoch(0)
+      delta.replay(record)
+      val leaderMetadataImage = imageFromTopics(delta.apply())

Review Comment:
   Could we inline this variable?



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