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


##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -27,9 +27,11 @@ import kafka.utils.{DelayedItem, Logging, Pool}
 import org.apache.kafka.common.errors._
 import org.apache.kafka.common.internals.PartitionStates
 import 
org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
+import org.apache.kafka.common.message.FetchResponseData.PartitionData
 import org.apache.kafka.common.message.{FetchResponseData, 
OffsetForLeaderEpochRequestData}
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Records}
+//import org.apache.kafka.common.requests.FetchRequest.PartitionData

Review Comment:
   Should we remove this import?



##########
core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala:
##########
@@ -633,13 +669,18 @@ class AbstractFetcherThreadTest {
       mkBatch(baseOffset = 7, leaderEpoch = 5, new SimpleRecord("h".getBytes)),
       mkBatch(baseOffset = 8, leaderEpoch = 5, new SimpleRecord("i".getBytes)))
 
-
     val leaderState = PartitionState(leaderLog, leaderEpoch = 5, highWatermark 
= 8L, rlmEnabled = true)
     // Overriding the log start offset to zero for mocking the scenario of 
segment 0-4 moved to remote store.
     leaderState.logStartOffset = 0
     fetcher.mockLeader.setLeaderState(partition, leaderState)
     
fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState)
 
+    def buildRemoteLog(topicPartition: TopicPartition, leaderLogStartOffset: 
Long): Unit = {
+      fetcher.truncateFullyAndStartAt(topicPartition, 
leaderState.localLogStartOffset)
+      replicaState.logStartOffset = leaderLogStartOffset

Review Comment:
   Well, `buildRemoteLog()` does two things (1) call 
`fetcher.truncateFullyAndStartAt` and (2) set `replicaState.logStartOffset`. 
For (1), since the truncation logic is moved to TierStateMachine, it probably 
should be done in `MockTierStateMachine.start()` directly. For (2), the 
existing test doesn't need to set `replicaState.logStartOffset.` So, it seems 
it's unnecessary? If address both (1) and (2), then the callback is not needed.



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -424,6 +406,13 @@ abstract class AbstractFetcherThread(name: String,
                   if (onPartitionFenced(topicPartition, 
fetchPartitionData.currentLeaderEpoch))
                     partitionsWithError += topicPartition
 
+                case Errors.OFFSET_MOVED_TO_TIERED_STORAGE =>
+                  debug(s"Received error 
${Errors.OFFSET_MOVED_TO_TIERED_STORAGE}, " +
+                    s"at fetch offset: ${currentFetchState.fetchOffset}, " + 
s"topic-partition: $topicPartition")
+                  if (!handleOffsetsMovedToTieredStorage(topicPartition, 
currentFetchState, fetchPartitionData.currentLeaderEpoch, partitionData)) {
+                    partitionsWithError += topicPartition
+                  }

Review Comment:
   No need for brackets for single line statements.



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -794,17 +742,18 @@ abstract class AbstractFetcherThread(name: String,
    *
    * @param topicPartition topic partition
    * @param fetchState current partition fetch state.
-   * @param leaderEpochInRequest current leader epoch sent in the fetch 
request.
-   * @param leaderLogStartOffset log-start-offset in the leader replica.
+   * @param fetchPartitionData the fetch request data for this topic partition

Review Comment:
   This is the fetch response data. Also, could we add the missing param?



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