chia7712 commented on code in PR #20009:
URL: https://github.com/apache/kafka/pull/20009#discussion_r2160425711


##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -1468,26 +1455,19 @@ class ReplicaManagerTest {
       leaderBrokerId, countDownLatch, expectTruncation = true, topicId = 
Optional.of(topicId))
 
     try {
-      val brokerList = Seq[Integer](0, 1).asJava
 
       val tp0 = new TopicPartition(topic, 0)
       val tidp0 = new TopicIdPartition(topicId, tp0)
 
       // Make this replica the follower
-      val leaderAndIsrRequest = 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,
-        Collections.singletonMap(topic, topicId),
-        Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
-      replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest, (_, _) => 
())
+      val followerDelta = createFollowerDelta(

Review Comment:
   you have create log first.
   ```scala
       val partition = replicaManager.createPartition(new TopicPartition(topic, 
0))
       partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
         new 
LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints.asJava), None)
   ```



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