Yunyung commented on code in PR #20014:
URL: https://github.com/apache/kafka/pull/20014#discussion_r2161654012


##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -312,38 +311,27 @@ class ReplicaManagerTest {
       alterPartitionManager = alterPartitionManager)
 
     try {
-      val partition = rm.createPartition(new TopicPartition(topic, 0))
-      partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
-        new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None)
+      val initialDelta = topicsCreateDelta(0, isStartIdLeader = true,
+        partitions = List(0), topicName = topic, topicId =  topicIds(topic))
+      val initialImage = imageFromTopics(initialDelta.apply())
+      rm.applyDelta(initialDelta, initialImage)
+      val partition = rm.getPartitionOrException(topicPartition)
 
-      rm.becomeLeaderOrFollower(0, new LeaderAndIsrRequest.Builder(0, 0, 
brokerEpoch,
-        Seq(new LeaderAndIsrRequest.PartitionState()
-          .setTopicName(topic)
-          .setPartitionIndex(0)
-          .setControllerEpoch(0)
-          .setLeader(0)
-          .setLeaderEpoch(0)
-          .setIsr(Seq[Integer](0).asJava)
-          .setPartitionEpoch(0)
-          .setReplicas(Seq[Integer](0).asJava)
-          .setIsNew(false)).asJava,
-        Collections.singletonMap(topic, topicId),
-        Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ())
-      appendRecords(rm, new TopicPartition(topic, 0),
+      appendRecords(rm,topicPartition,

Review Comment:
   ```suggestion
         appendRecords(rm, topicPartition,
   ```



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -1893,13 +1859,11 @@ class ReplicaManagerTest {
 
     val replicaManager = 
setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager,
 List(tp0, tp1))
     try {
-      replicaManager.becomeLeaderOrFollower(1,
-        makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new 
LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
-        (_, _) => ())
-
-      replicaManager.becomeLeaderOrFollower(1,
-        makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), new 
LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
-        (_, _) => ())
+      // Step 1: Apply KRaft metadata to assign tp0 and tp1 as leaders
+      val directoryIds = replicaManager.logManager.directoryIdsSet.toList
+      val leaderDelta = topicsCreateDelta(startId = 0, isStartIdLeader = true, 
partitions = List(0, 1), directoryIds = directoryIds, topic, topicIds(topic))

Review Comment:
   Fix the test:
   ```suggestion
         val leaderDelta = topicsCreateDelta(startId = 0, isStartIdLeader = 
true, partitions = List(0, 1), List.empty, topic, topicIds(topic))
   ```
   



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -312,38 +311,27 @@ class ReplicaManagerTest {
       alterPartitionManager = alterPartitionManager)
 
     try {
-      val partition = rm.createPartition(new TopicPartition(topic, 0))
-      partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
-        new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None)
+      val initialDelta = topicsCreateDelta(0, isStartIdLeader = true,
+        partitions = List(0), topicName = topic, topicId =  topicIds(topic))
+      val initialImage = imageFromTopics(initialDelta.apply())

Review Comment:
   Why call it initial? I think you copied it from other test cases you 
modified.  I'd prefer consistent naming in this file for this test case.



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