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


##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -785,17 +732,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:
   What is the returned boolean indicating?



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

Review Comment:
   Have you considered refactoring this method to ease testability? This could 
be part of another PR too to avoid changing too many parts in one code refactor.



##########
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.
+                if (earlierEpochEndOffset.endOffset() > 
previousOffsetToLeaderLocalLogStartOffset) {
+                    // Always use the leader epoch from returned 
earlierEpochEndOffset.
+                    // This gives the respective leader epoch, that will 
handle any gaps in epochs.
+                    // For ex, leader epoch cache contains:
+                    // leader-epoch   start-offset
+                    //  0               20
+                    //  1               85
+                    //  <2> - gap no messages were appended in this leader 
epoch.
+                    //  3               90
+                    //  4               98
+                    // There is a gap in leader epoch. For 
leaderLocalLogStartOffset as 90, leader-epoch is 3.
+                    // fetchEarlierEpochEndOffset(2) will return leader-epoch 
as 1, end-offset as 90.
+                    // So, for offset 89, we should return leader epoch as 1 
like below.
+                    targetEpoch = earlierEpochEndOffset.leaderEpoch();
+                } else {
+                    targetEpoch = epochForLeaderLocalLogStartOffset;
+                }
+            }
+
+            Optional<RemoteLogSegmentMetadata> maybeRlsm = 
rlm.fetchRemoteLogSegmentMetadata(topicPartition, targetEpoch, 
previousOffsetToLeaderLocalLogStartOffset);
+
+            if (maybeRlsm.isPresent()) {

Review Comment:
   Note: if the rlmMetadata is unavailable for an extended period of time, the 
replica fetcher will keep retrying indefinitely to construct the starting fetch 
state for the partition. This will lead to an `OffsetForLeaderEpoch` and 
`ListOffsets` requests every time. If a large number of partitions are 
impacted, that will generate unnecessary inter-broker traffic on the cluster - 
although marginal most of the time. As an optimization, we could store the 
leader epoch associated to the leader's local log start offset - 1 which was 
retrieved here (we would still, however, need to query for the local log start 
offset on the leader on every iteration).
   
   The asynchronous resolution of the correct fetch state from the remote 
storages (KAFKA-13560) will prevent the extra load on the replica fetcher 
thread itself. The consideration above applies to the RPCs which are made on 
the synchronous fetch path.



##########
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<>();

Review Comment:
   nit - define the type of `partitionsWithEpoch` as `Map`.



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