Yunyung commented on code in PR #20014: URL: https://github.com/apache/kafka/pull/20014#discussion_r2161736851
########## core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala: ########## @@ -1251,52 +1222,45 @@ class ReplicaManagerTest { */ private def verifyBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate(extraProps: Properties, expectTruncation: Boolean): Unit = { - val topicPartition = 0 val followerBrokerId = 0 val leaderBrokerId = 1 - val controllerId = 0 - val controllerEpoch = 0 var leaderEpoch = 1 val leaderEpochIncrement = 2 - val aliveBrokerIds = Seq[Integer](followerBrokerId, leaderBrokerId) val countDownLatch = new CountDownLatch(1) val offsetFromLeader = 5 - // Prepare the mocked components for the test val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(new MockTimer(time), - topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, + 0, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = expectTruncation, localLogOffset = Optional.of(10), offsetFromLeader = offsetFromLeader, extraProps = extraProps, topicId = Optional.of(topicId)) try { // Initialize partition state to follower, with leader = 1, leaderEpoch = 1 - val tp = new TopicPartition(topic, topicPartition) - val partition = replicaManager.createPartition(tp) - val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints.asJava) - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) - partition.makeFollower( - leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds), - offsetCheckpoints, - None) + val directoryIds = replicaManager.logManager.directoryIdsSet.toList + val initialFollowerDelta = topicsCreateDelta( startId = followerBrokerId, isStartIdLeader = false, partitions = List(0), directoryIds = directoryIds, topic, topicIds(topic), leaderEpoch) + val initialImage = imageFromTopics(initialFollowerDelta.apply()) Review Comment: This should fix the test. You can update based on this. ```suggestion val tp = new TopicPartition(topic, topicPartition) val partition = replicaManager.createPartition(tp) val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints.asJava) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) val initialFollowerDelta = topicsCreateDelta(startId = followerBrokerId, isStartIdLeader = false, partitions = List(0), List.empty, topic, topicIds(topic), leaderEpoch) val initialImage = imageFromTopics(initialFollowerDelta.apply()) replicaManager.applyDelta(initialFollowerDelta, initialImage) ``` -- 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