You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "junrao (via GitHub)" <gi...@apache.org> on 2023/02/08 19:48:37 UTC

[GitHub] [kafka] junrao commented on a diff in pull request #13206: [KAFKA-14685] Refactor logic to handle OFFSET_MOVED_TO_TIERED_STORAGE error

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


##########
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+/**
+ The replica fetcher tier state machine follows a state machine progression.
+
+ Currently, the tier state machine follows a synchronous execution, and we only need to start the machine.
+ There is no need to advance the state.
+
+ When started, the tier state machine will fetch the local log start offset of the
+ leader and then build the follower's remote log aux state until the leader's
+ local log start offset.
+ */
+public class ReplicaFetcherTierStateMachine implements TierStateMachine {
+    private static final Logger log = LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class);

Review Comment:
   This seems a bit weird. The log typically is an instance level object.



##########
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+/**
+ The replica fetcher tier state machine follows a state machine progression.
+
+ Currently, the tier state machine follows a synchronous execution, and we only need to start the machine.
+ There is no need to advance the state.
+
+ When started, the tier state machine will fetch the local log start offset of the
+ leader and then build the follower's remote log aux state until the leader's
+ local log start offset.
+ */
+public class ReplicaFetcherTierStateMachine implements TierStateMachine {
+    private static final Logger log = LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class);
+
+    private LeaderEndPoint leader;
+    private ReplicaManager replicaMgr;
+    private Integer fetchBackOffMs;
+
+    public ReplicaFetcherTierStateMachine(LeaderEndPoint leader,
+                                          ReplicaManager replicaMgr) {
+        this.leader = leader;
+        this.replicaMgr = replicaMgr;
+    }
+
+
+    /**
+     * Start the tier state machine for the provided topic partition. Currently, this start method will build the
+     * entire remote aux log state synchronously.
+     *
+     * @param topicPartition the topic partition
+     * @param currentFetchState the current PartitionFetchState which will
+     *                          be used to derive the return value
+     * @param fetchPartitionData the data from the fetch response that returned the offset moved to tiered storage error
+     *
+     * @return the new PartitionFetchState after the successful start of the
+     *         tier state machine
+     */
+    public PartitionFetchState start(TopicPartition topicPartition,
+                                     PartitionFetchState currentFetchState,
+                                     PartitionData fetchPartitionData) throws Exception {
+
+        Tuple2<Object, Object> epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch());
+        int epoch = (int) epochAndLeaderLocalStartOffset._1;
+        long leaderLocalStartOffset = (long) epochAndLeaderLocalStartOffset._2;
+
+        long offsetToFetch = buildRemoteLogAuxState(topicPartition, currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, fetchPartitionData.logStartOffset);
+
+        Tuple2<Object, Object> fetchLatestOffsetResult = leader.fetchLatestOffset(topicPartition, currentFetchState.currentLeaderEpoch());
+        long leaderEndOffset = (long) fetchLatestOffsetResult._2;
+
+        long initialLag = leaderEndOffset - offsetToFetch;
+
+        return PartitionFetchState.apply(currentFetchState.topicId(), offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(),
+                Fetching$.MODULE$, replicaMgr.localLogOrException(topicPartition).latestEpoch());
+    }
+
+    /**
+     * This is currently a no-op but will be used for implementing async tiering logic in KAFKA-13560.
+     *
+     * @param topicPartition the topic partition
+     * @param currentFetchState the current PartitionFetchState which will
+     *                          be used to derive the return value
+     *
+     * @return the original PartitionFetchState
+     */
+    public Optional<PartitionFetchState> maybeAdvanceState(TopicPartition topicPartition,
+                                                           PartitionFetchState currentFetchState) {
+        // No-op for now
+        return Optional.of(currentFetchState);
+    }
+
+    private EpochEndOffset fetchEarlierEpochEndOffset(Integer epoch,
+                                                      TopicPartition partition,
+                                                      Integer currentLeaderEpoch) {
+        int previousEpoch = epoch - 1;
+
+        // Find the end-offset for the epoch earlier to the given epoch from the leader
+        HashMap<TopicPartition, OffsetForLeaderPartition> partitionsWithEpochs = new HashMap<>();
+        partitionsWithEpochs.put(partition, new OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch));
+
+        Option<EpochEndOffset> maybeEpochEndOffset = leader.fetchEpochEndOffsets(JavaConverters.asScala(partitionsWithEpochs)).get(partition);
+        if (maybeEpochEndOffset.isEmpty()) {
+            throw new KafkaException("No response received for partition: " + partition);
+        }
+
+        EpochEndOffset epochEndOffset = maybeEpochEndOffset.get();
+        if (epochEndOffset.errorCode() != Errors.NONE.code()) {
+            throw Errors.forCode(epochEndOffset.errorCode()).exception();
+        }
+
+        return epochEndOffset;
+    }
+
+    private List<EpochEntry> readLeaderEpochCheckpoint(RemoteLogManager rlm,
+                                                       RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, RemoteStorageException {
+        InputStream inputStream = rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH);
+        try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) {
+            CheckpointFile.CheckpointReadBuffer<EpochEntry> readBuffer = new CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, LeaderEpochCheckpointFile.FORMATTER);
+            return readBuffer.read();
+        }
+    }
+
+    private void buildProducerSnapshotFile(File snapshotFile,
+                                           RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                           RemoteLogManager rlm) throws IOException, RemoteStorageException {
+        File tmpSnapshotFile = new File(snapshotFile.getAbsolutePath() + ".tmp");
+        // Copy it to snapshot file in atomic manner.
+        Files.copy(rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT),
+                tmpSnapshotFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+        Utils.atomicMoveWithFallback(tmpSnapshotFile.toPath(), snapshotFile.toPath(), false);
+    }
+
+    /**
+     * It tries to build the required state for this partition from leader and remote storage so that it can start
+     * fetching records from the leader.
+     */
+    private Long buildRemoteLogAuxState(TopicPartition topicPartition,
+                                        Integer currentLeaderEpoch,
+                                        Long leaderLocalLogStartOffset,
+                                        Integer epochForLeaderLocalLogStartOffset,
+                                        Long leaderLogStartOffset) throws IOException, RemoteStorageException {
+
+        UnifiedLog log = replicaMgr.localLogOrException(topicPartition);
+
+        long nextOffset;
+
+        if (log.remoteStorageSystemEnable() && log.config().remoteLogConfig.remoteStorageEnable) {
+            if (replicaMgr.remoteLogManager().isEmpty()) throw new IllegalStateException("RemoteLogManager is not yet instantiated");
+
+            RemoteLogManager rlm = replicaMgr.remoteLogManager().get();
+
+            // Find the respective leader epoch for (leaderLocalLogStartOffset - 1). We need to build the leader epoch cache
+            // until that offset
+            long previousOffsetToLeaderLocalLogStartOffset = leaderLocalLogStartOffset - 1;
+            int targetEpoch;
+            // If the existing epoch is 0, no need to fetch from earlier epoch as the desired offset(leaderLogStartOffset - 1)
+            // will have the same epoch.
+            if (epochForLeaderLocalLogStartOffset == 0) {
+                targetEpoch = epochForLeaderLocalLogStartOffset;
+            } else {
+                // Fetch the earlier epoch/end-offset(exclusive) from the leader.
+                EpochEndOffset earlierEpochEndOffset = fetchEarlierEpochEndOffset(epochForLeaderLocalLogStartOffset, topicPartition, currentLeaderEpoch);
+                // Check if the target offset lies with in the range of earlier epoch. Here, epoch's end-offset is exclusive.

Review Comment:
   This is an existing issue. But change "with in" to "within"?



##########
core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala:
##########
@@ -1334,6 +1397,31 @@ class AbstractFetcherThreadTest {
     }
   }
 
+  class MockTierStateMachine(leader: LeaderEndPoint) extends ReplicaFetcherTierStateMachine(leader, null) {
+
+    var startCallback: (TopicPartition, Long) => Unit = (_,_) => {}

Review Comment:
   Could this be private?



##########
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+/**
+ The replica fetcher tier state machine follows a state machine progression.
+
+ Currently, the tier state machine follows a synchronous execution, and we only need to start the machine.
+ There is no need to advance the state.
+
+ When started, the tier state machine will fetch the local log start offset of the
+ leader and then build the follower's remote log aux state until the leader's
+ local log start offset.
+ */
+public class ReplicaFetcherTierStateMachine implements TierStateMachine {
+    private static final Logger log = LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class);
+
+    private LeaderEndPoint leader;
+    private ReplicaManager replicaMgr;
+    private Integer fetchBackOffMs;
+
+    public ReplicaFetcherTierStateMachine(LeaderEndPoint leader,
+                                          ReplicaManager replicaMgr) {
+        this.leader = leader;
+        this.replicaMgr = replicaMgr;
+    }
+
+
+    /**
+     * Start the tier state machine for the provided topic partition. Currently, this start method will build the
+     * entire remote aux log state synchronously.
+     *
+     * @param topicPartition the topic partition
+     * @param currentFetchState the current PartitionFetchState which will
+     *                          be used to derive the return value
+     * @param fetchPartitionData the data from the fetch response that returned the offset moved to tiered storage error
+     *
+     * @return the new PartitionFetchState after the successful start of the
+     *         tier state machine
+     */
+    public PartitionFetchState start(TopicPartition topicPartition,
+                                     PartitionFetchState currentFetchState,
+                                     PartitionData fetchPartitionData) throws Exception {
+
+        Tuple2<Object, Object> epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch());
+        int epoch = (int) epochAndLeaderLocalStartOffset._1;
+        long leaderLocalStartOffset = (long) epochAndLeaderLocalStartOffset._2;
+
+        long offsetToFetch = buildRemoteLogAuxState(topicPartition, currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, fetchPartitionData.logStartOffset);
+
+        Tuple2<Object, Object> fetchLatestOffsetResult = leader.fetchLatestOffset(topicPartition, currentFetchState.currentLeaderEpoch());
+        long leaderEndOffset = (long) fetchLatestOffsetResult._2;
+
+        long initialLag = leaderEndOffset - offsetToFetch;
+
+        return PartitionFetchState.apply(currentFetchState.topicId(), offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(),
+                Fetching$.MODULE$, replicaMgr.localLogOrException(topicPartition).latestEpoch());
+    }
+
+    /**
+     * This is currently a no-op but will be used for implementing async tiering logic in KAFKA-13560.
+     *
+     * @param topicPartition the topic partition
+     * @param currentFetchState the current PartitionFetchState which will
+     *                          be used to derive the return value
+     *
+     * @return the original PartitionFetchState
+     */
+    public Optional<PartitionFetchState> maybeAdvanceState(TopicPartition topicPartition,
+                                                           PartitionFetchState currentFetchState) {
+        // No-op for now
+        return Optional.of(currentFetchState);
+    }
+
+    private EpochEndOffset fetchEarlierEpochEndOffset(Integer epoch,
+                                                      TopicPartition partition,
+                                                      Integer currentLeaderEpoch) {
+        int previousEpoch = epoch - 1;
+
+        // Find the end-offset for the epoch earlier to the given epoch from the leader
+        HashMap<TopicPartition, OffsetForLeaderPartition> partitionsWithEpochs = new HashMap<>();
+        partitionsWithEpochs.put(partition, new OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch));
+
+        Option<EpochEndOffset> maybeEpochEndOffset = leader.fetchEpochEndOffsets(JavaConverters.asScala(partitionsWithEpochs)).get(partition);
+        if (maybeEpochEndOffset.isEmpty()) {
+            throw new KafkaException("No response received for partition: " + partition);
+        }
+
+        EpochEndOffset epochEndOffset = maybeEpochEndOffset.get();
+        if (epochEndOffset.errorCode() != Errors.NONE.code()) {
+            throw Errors.forCode(epochEndOffset.errorCode()).exception();
+        }
+
+        return epochEndOffset;
+    }
+
+    private List<EpochEntry> readLeaderEpochCheckpoint(RemoteLogManager rlm,
+                                                       RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, RemoteStorageException {
+        InputStream inputStream = rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH);
+        try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) {
+            CheckpointFile.CheckpointReadBuffer<EpochEntry> readBuffer = new CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, LeaderEpochCheckpointFile.FORMATTER);
+            return readBuffer.read();
+        }
+    }
+
+    private void buildProducerSnapshotFile(File snapshotFile,
+                                           RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                           RemoteLogManager rlm) throws IOException, RemoteStorageException {
+        File tmpSnapshotFile = new File(snapshotFile.getAbsolutePath() + ".tmp");
+        // Copy it to snapshot file in atomic manner.
+        Files.copy(rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT),
+                tmpSnapshotFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+        Utils.atomicMoveWithFallback(tmpSnapshotFile.toPath(), snapshotFile.toPath(), false);
+    }
+
+    /**
+     * It tries to build the required state for this partition from leader and remote storage so that it can start
+     * fetching records from the leader.
+     */

Review Comment:
   This is an existing issue, but could we add a comment describing the return value?



##########
core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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;
+
+import kafka.cluster.Partition;
+import kafka.log.LeaderOffsetIncremented$;
+import kafka.log.UnifiedLog;
+import kafka.log.remote.RemoteLogManager;
+import kafka.server.checkpoints.LeaderEpochCheckpointFile;
+import kafka.server.epoch.EpochEntry;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
+import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+import scala.collection.immutable.Seq;
+
+/**
+ The replica fetcher tier state machine follows a state machine progression.
+
+ Currently, the tier state machine follows a synchronous execution and only the start is needed.
+ There is no need to advance the state.
+
+ When started, the tier state machine will fetch the local log start offset of the
+ leader and then build the follower's remote log aux state until the leader's
+ local log start offset.
+ */
+public class ReplicaFetcherTierStateMachine implements TierStateMachine {
+    private static final Logger LOGGER = LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class);
+
+    private LeaderEndPoint leader;
+    private ReplicaManager replicaMgr;
+    private Integer fetchBackOffMs;

Review Comment:
   Has this been removed?



##########
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:
   Here, we want to update the replicaState in the MockFetcherThread. Would it be better to call fetcher.replicaPartitionState to get the replicaState explicitly?
   
   Also, adding a callback in a Mock class seems a bit complicated. Alternatively, we could override `doWork`() in MockFetcherThread so that it updates its replicaPartitionState from the partitionState in AbstractFetcherThread after each call. This way, we could get rid of the callback in mockTierStateMachine.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org