junrao commented on code in PR #13206:
URL: https://github.com/apache/kafka/pull/13206#discussion_r1116386328


##########
core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala:
##########
@@ -615,14 +649,29 @@ class AbstractFetcherThreadTest {
   @Test
   def testFollowerFetchMovedToTieredStore(): Unit = {
     val partition = new TopicPartition("topic", 0)
-    val fetcher = new MockFetcherThread(new MockLeaderEndPoint)
 
     val replicaLog = Seq(
       mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)),
       mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
       mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
 
     val replicaState = PartitionState(replicaLog, leaderEpoch = 5, 
highWatermark = 0L, rlmEnabled = true)
+
+    val mockLeaderEndpoint = new MockLeaderEndPoint
+    val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) {
+      // override the start() of MockTierStateMachine to mimic 
truncateFullyAndStartAt and update the replicaState in the MockFetcherThread
+      override def start(topicPartition: TopicPartition,
+                         currentFetchState: PartitionFetchState,
+                         fetchPartitionData: FetchResponseData.PartitionData): 
PartitionFetchState = {
+        replicaState.log.clear()
+        replicaState.localLogStartOffset = 5
+        replicaState.logEndOffset = 5

Review Comment:
   Instead of the customizing for this test, I am wondering if we could make 
this more general. For example, could we pass in `fetcher` to 
MockTierStateMachine and call `fetcher.truncateFullyAndStartAt() `in 
`MockTierStateMachine.start()`.



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