junrao commented on code in PR #13206: URL: https://github.com/apache/kafka/pull/13206#discussion_r1116408589
########## core/src/main/java/kafka/server/TierStateMachine.java: ########## @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server; Review Comment: Should this be in the storage module under org.apache.kafka.server.log.remote.storage package? Eventually, we could move ReplicaFetcherTierStateMachine to the storage module too. ########## 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: Got it. So, if we do that, it's the same as the callback logic you had earlier. Here is another way to improve this. Since AbstractFetcherThread already exposes the `partitionState` through `fetchState()`, we could just get rid of `replicaPartitionStates` in MockFetcherThread and purely rely on the `partitionState` that's changed on every `doWork()` call. Will that work? Since that's a bigger change, we could probably just take the callback code you had in this PR and make the bigger change in a separate PR. -- 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