You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/04/05 18:43:43 UTC

[GitHub] [kafka] junrao commented on a change in pull request #10218: KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606343187



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be

Review comment:
       indicate => indicates Ditto in a few other places.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {

Review comment:
       I guess this is an internal class. Will this be exposed in javadoc since currently it includes **/org/apache/kafka/server/log/remote/storage/* ?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no
+        // need to add for that state here.
+        return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED)
+               ? Optional.empty() : Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentId remoteLogSegmentId = metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for :" +
+                                                      remoteLogSegmentId);
+        }
+
+        // Check the state transition.
+        checkStateTransition(existingMetadata.state(), targetState);
+
+        switch (targetState) {
+            case COPY_SEGMENT_STARTED:
+                // Callers should use addCopyInProgressSegment to add RemoteLogSegmentMetadata with state as
+                // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+                throw new IllegalArgumentException("Metadata with " + RemoteLogSegmentState.COPY_SEGMENT_STARTED +

Review comment:
       Should we include metadataUpdate in the message of the exception?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogLeaderEpochState.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * This class represents the in-memory state of segments associated with a leader epoch. This includes the mapping of offset to
+ * segment ids and unreferenced segments which are not mapped to any offset but they exist in remote storage.
+ * <p>
+ * This is used by {@link RemoteLogMetadataCache} to track the segments for each leader epoch.
+ */
+class RemoteLogLeaderEpochState {
+
+    // It contains offset to segment ids mapping with the segment state as COPY_SEGMENT_FINISHED.
+    private final NavigableMap<Long, RemoteLogSegmentId> offsetToId = new ConcurrentSkipListMap<>();
+
+    /**
+     * It represents unreferenced segments for this leader epoch. It contains the segments still in COPY_SEGMENT_STARTED
+     * and DELETE_SEGMENT_STARTED state or these have been replaced by callers with other segments having the same
+     * start offset for the leader epoch. These will be returned by {@link RemoteLogMetadataCache#listAllRemoteLogSegments()}
+     * and {@link RemoteLogMetadataCache#listRemoteLogSegments(int leaderEpoch)} so that callers can clean them up if
+     * they still exist. These will be cleaned from the cache once they reach DELETE_SEGMENT_FINISHED state.
+     */
+    private final Set<RemoteLogSegmentId> unreferencedSegmentIds = ConcurrentHashMap.newKeySet();
+
+    // It represents the highest log offset of the segments that were updated with updateHighestLogOffset.
+    private volatile Long highestLogOffset;
+
+    /**
+     * Returns all the segments associated with this leader epoch sorted by start offset in ascending order.
+     *
+     * @param idToSegmentMetadata mapping of id to segment metadata. This will be used to get RemoteLogSegmentMetadata
+     *                            for an id to be used for sorting.
+     * @return
+     */
+    Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments(Map<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata) {
+        // Return all the segments including unreferenced metadata.
+        int size = offsetToId.size() + unreferencedSegmentIds.size();
+        if (size == 0) {
+            return Collections.emptyIterator();
+        }
+
+        ArrayList<RemoteLogSegmentMetadata> metadataList = new ArrayList<>(size);
+        for (RemoteLogSegmentId id : offsetToId.values()) {
+            metadataList.add(idToSegmentMetadata.get(id));
+        }
+
+        if (!unreferencedSegmentIds.isEmpty()) {
+            for (RemoteLogSegmentId id : unreferencedSegmentIds) {
+                metadataList.add(idToSegmentMetadata.get(id));
+            }
+
+            // sort only when unreferenced entries exist as they are already sorted in offsetToId.
+            metadataList.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        }
+
+        return metadataList.iterator();
+    }
+
+    void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                   Long leaderEpochEndOffset) {
+        // Add the segment epochs mapping as the segment is copied successfully.
+        RemoteLogSegmentId oldEntry = offsetToId.put(startOffset, remoteLogSegmentId);
+
+        // Remove the metadata from unreferenced entries as it is successfully copied and added to the offset mapping.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // Add the old entry to unreferenced entries as the mapping is removed for the old entry.
+        if (oldEntry != null) {
+            unreferencedSegmentIds.add(oldEntry);

Review comment:
       Hmm, during unclean leader election, some of the old segments may need to be added to unreferenced segment id list but may not have the exact offset of the new segment. How are those segments handled here?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>

Review comment:
       The following table is a bit hard to read for developers. Since this is not meant for a public interface, could we make it more readable for developers?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no

Review comment:
       It's weird to reference offsetToId here since it's in a separate class.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no
+        // need to add for that state here.
+        return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED)
+               ? Optional.empty() : Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentId remoteLogSegmentId = metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for :" +
+                                                      remoteLogSegmentId);
+        }
+
+        // Check the state transition.
+        checkStateTransition(existingMetadata.state(), targetState);
+
+        switch (targetState) {
+            case COPY_SEGMENT_STARTED:
+                // Callers should use addCopyInProgressSegment to add RemoteLogSegmentMetadata with state as
+                // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+                throw new IllegalArgumentException("Metadata with " + RemoteLogSegmentState.COPY_SEGMENT_STARTED +
+                                                   " can not be updated");
+            case COPY_SEGMENT_FINISHED:
+                handleSegmentWithCopySegmentFinishedState(metadataUpdate, existingMetadata);
+                break;
+            case DELETE_SEGMENT_STARTED:
+                handleSegmentWithDeleteSegmentStartedState(metadataUpdate, existingMetadata);
+                break;
+            case DELETE_SEGMENT_FINISHED:
+                handleSegmentWithDeleteSegmentFinishedState(metadataUpdate, existingMetadata);
+                break;
+            default:
+                throw new IllegalArgumentException("Metadata with the state" + targetState + " is not supported");
+        }
+    }
+
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadataUpdate metadataUpdate,
+                                                           RemoteLogSegmentMetadata existingMetadata) {
+        log.debug("Adding remote log segment metadata to leader epoch mappings with update: [{}]", metadataUpdate);
+
+        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
+                RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState);
+
+        // Put the entry with the updated metadata.
+        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
+                existingMetadata.createWithUpdates(metadataUpdate));
+    }
+
+    private void handleSegmentWithDeleteSegmentStartedState(RemoteLogSegmentMetadataUpdate metadataUpdate,
+                                                            RemoteLogSegmentMetadata existingMetadata) {
+        log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+
+        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
+                RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentStartedState);
+
+        // Put the entry with the updated metadata.
+        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
+                existingMetadata.createWithUpdates(metadataUpdate));
+    }
+
+    private void handleSegmentWithDeleteSegmentFinishedState(RemoteLogSegmentMetadataUpdate metadataUpdate,
+                                                             RemoteLogSegmentMetadata existingMetadata) {
+        log.debug("Removing the entry as it reached the terminal state: [{}]", metadataUpdate);
+
+        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
+                RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentFinishedState);
+
+        // Remove the segment's id to metadata mapping because this segment is considered as deleted and it cleared all
+        // the state of this segment in the cache.
+        idToSegmentMetadata.remove(existingMetadata.remoteLogSegmentId());
+    }
+
+    private void doHandleSegmentStateTransitionForLeaderEpochs(RemoteLogSegmentMetadata existingMetadata,
+                                                               RemoteLogLeaderEpochState.Action action) {
+        RemoteLogSegmentId remoteLogSegmentId = existingMetadata.remoteLogSegmentId();
+        Map<Integer, Long> leaderEpochToOffset = existingMetadata.segmentLeaderEpochs();
+
+        // Go through all the leader epochs and apply the given action.
+        for (Map.Entry<Integer, Long> entry : leaderEpochToOffset.entrySet()) {
+            Integer leaderEpoch = entry.getKey();
+            Long startOffset = entry.getValue();
+            RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+            if (remoteLogLeaderEpochState == null) {
+                throw new IllegalStateException("RemoteLogLeaderEpochState does not exist for the leader epoch: "
+                                                + leaderEpoch);
+            } else {
+                long leaderEpochEndOffset = highestOffsetForEpoch(leaderEpoch, existingMetadata);
+                action.accept(remoteLogLeaderEpochState, startOffset, remoteLogSegmentId, leaderEpochEndOffset);
+            }
+        }
+    }
+
+    private long highestOffsetForEpoch(Integer leaderEpoch, RemoteLogSegmentMetadata segmentMetadata) {
+        //compute the highest offset for the leader epoch with in the segment
+        NavigableMap<Integer, Long> epochToOffset = segmentMetadata.segmentLeaderEpochs();
+        Map.Entry<Integer, Long> nextEntry = epochToOffset.higherEntry(leaderEpoch);
+
+        // Update with the given metadata for leader epoch

Review comment:
       This comment is confusing since there is no update here.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no
+        // need to add for that state here.
+        return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED)
+               ? Optional.empty() : Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentId remoteLogSegmentId = metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for :" +
+                                                      remoteLogSegmentId);
+        }
+
+        // Check the state transition.
+        checkStateTransition(existingMetadata.state(), targetState);
+
+        switch (targetState) {
+            case COPY_SEGMENT_STARTED:
+                // Callers should use addCopyInProgressSegment to add RemoteLogSegmentMetadata with state as
+                // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+                throw new IllegalArgumentException("Metadata with " + RemoteLogSegmentState.COPY_SEGMENT_STARTED +
+                                                   " can not be updated");
+            case COPY_SEGMENT_FINISHED:
+                handleSegmentWithCopySegmentFinishedState(metadataUpdate, existingMetadata);
+                break;
+            case DELETE_SEGMENT_STARTED:
+                handleSegmentWithDeleteSegmentStartedState(metadataUpdate, existingMetadata);
+                break;
+            case DELETE_SEGMENT_FINISHED:
+                handleSegmentWithDeleteSegmentFinishedState(metadataUpdate, existingMetadata);
+                break;
+            default:
+                throw new IllegalArgumentException("Metadata with the state" + targetState + " is not supported");
+        }
+    }
+
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadataUpdate metadataUpdate,
+                                                           RemoteLogSegmentMetadata existingMetadata) {
+        log.debug("Adding remote log segment metadata to leader epoch mappings with update: [{}]", metadataUpdate);
+
+        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
+                RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState);
+
+        // Put the entry with the updated metadata.
+        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
+                existingMetadata.createWithUpdates(metadataUpdate));
+    }
+
+    private void handleSegmentWithDeleteSegmentStartedState(RemoteLogSegmentMetadataUpdate metadataUpdate,
+                                                            RemoteLogSegmentMetadata existingMetadata) {
+        log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+
+        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
+                RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentStartedState);
+
+        // Put the entry with the updated metadata.
+        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
+                existingMetadata.createWithUpdates(metadataUpdate));
+    }
+
+    private void handleSegmentWithDeleteSegmentFinishedState(RemoteLogSegmentMetadataUpdate metadataUpdate,
+                                                             RemoteLogSegmentMetadata existingMetadata) {
+        log.debug("Removing the entry as it reached the terminal state: [{}]", metadataUpdate);
+
+        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
+                RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentFinishedState);
+
+        // Remove the segment's id to metadata mapping because this segment is considered as deleted and it cleared all
+        // the state of this segment in the cache.
+        idToSegmentMetadata.remove(existingMetadata.remoteLogSegmentId());
+    }
+
+    private void doHandleSegmentStateTransitionForLeaderEpochs(RemoteLogSegmentMetadata existingMetadata,
+                                                               RemoteLogLeaderEpochState.Action action) {
+        RemoteLogSegmentId remoteLogSegmentId = existingMetadata.remoteLogSegmentId();
+        Map<Integer, Long> leaderEpochToOffset = existingMetadata.segmentLeaderEpochs();
+
+        // Go through all the leader epochs and apply the given action.
+        for (Map.Entry<Integer, Long> entry : leaderEpochToOffset.entrySet()) {
+            Integer leaderEpoch = entry.getKey();
+            Long startOffset = entry.getValue();
+            RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+            if (remoteLogLeaderEpochState == null) {
+                throw new IllegalStateException("RemoteLogLeaderEpochState does not exist for the leader epoch: "
+                                                + leaderEpoch);
+            } else {
+                long leaderEpochEndOffset = highestOffsetForEpoch(leaderEpoch, existingMetadata);
+                action.accept(remoteLogLeaderEpochState, startOffset, remoteLogSegmentId, leaderEpochEndOffset);
+            }
+        }
+    }
+
+    private long highestOffsetForEpoch(Integer leaderEpoch, RemoteLogSegmentMetadata segmentMetadata) {
+        //compute the highest offset for the leader epoch with in the segment
+        NavigableMap<Integer, Long> epochToOffset = segmentMetadata.segmentLeaderEpochs();
+        Map.Entry<Integer, Long> nextEntry = epochToOffset.higherEntry(leaderEpoch);
+
+        // Update with the given metadata for leader epoch
+        //  - If there is no highest entry  OR
+        //  - If the existing entry's endOffset is lower than the given metadata's endOffset.
+        return nextEntry != null ? nextEntry.getValue() - 1 : segmentMetadata.endOffset();
+    }
+
+    /**
+     * Returns all the segments stored in this cache.
+     *
+     * @return
+     */
+    public Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments() {
+        // Return all the segments including unreferenced metadata.
+        return Collections.unmodifiableCollection(idToSegmentMetadata.values()).iterator();
+    }
+
+    /**
+     * Returns all the segments mapped to the leader epoch that exist in this cache sorted by {@link RemoteLogSegmentMetadata#startOffset()}.
+     *
+     * @param leaderEpoch leader epoch.
+     * @return
+     */
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(int leaderEpoch) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+        if (remoteLogLeaderEpochState == null) {
+            return Collections.emptyIterator();
+        }
+
+        return remoteLogLeaderEpochState.listAllRemoteLogSegments(idToSegmentMetadata);
+    }
+
+    /**
+     * Returns the highest offset of a segment for the given leader epoch if exists, else it returns empty. The segments
+     * that have reached the {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} or later states are considered here.
+     *
+     * @param leaderEpoch leader epoch
+     * @return
+     */
+    public Optional<Long> highestOffsetForEpoch(int leaderEpoch) {
+        RemoteLogLeaderEpochState entry = leaderEpochEntries.get(leaderEpoch);
+        return entry != null ? Optional.ofNullable(entry.highestLogOffset()) : Optional.empty();
+    }
+
+    /**
+     * This method tracks the given remote segment as not yet available for reads. It does not add the segment
+     * leader epoch offset mapping until this segment reaches COPY_SEGMENT_FINISHED state.
+     *
+     * @param remoteLogSegmentMetadata
+     */
+    public void addCopyInProgressSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        log.debug("Adding to in-progress state: [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED)
+        // but not to update the existing remote log segment metadata.
+        if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            throw new IllegalArgumentException(
+                    "Given remoteLogSegmentMetadata should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED

Review comment:
       Could we include remoteLogSegmentMetadata in the exception message?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache = new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        idToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache())
+                .addCopyInProgressSegment(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteStorageException {
+        log.debug("Updating remote log segment: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        getRemoteLogMetadataCache(metadataUpdate.remoteLogSegmentId().topicIdPartition())
+                .updateRemoteLogSegmentMetadata(metadataUpdate);
+    }
+
+    private RemoteLogMetadataCache getRemoteLogMetadataCache(TopicIdPartition topicIdPartition)
+            throws RemoteResourceNotFoundException {
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No existing metadata found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache;
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       int epochForOffset,
+                                                                       long offset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition,
+                                                int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).highestOffsetForEpoch(leaderEpoch);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+
+        RemotePartitionDeleteState targetState = remotePartitionDeleteMetadata.state();
+        RemotePartitionDeleteMetadata existingMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        RemotePartitionDeleteState existingState = existingMetadata != null ? existingMetadata.state() : null;
+        if (!RemotePartitionDeleteState.isValidTransition(existingState, targetState)) {
+            throw new IllegalStateException("Current state: " + existingState + ", target state: " + targetState);

Review comment:
       Since existingState can be null, we want to handle it properly.

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache = new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        idToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache())
+                .addCopyInProgressSegment(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteStorageException {
+        log.debug("Updating remote log segment: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        getRemoteLogMetadataCache(metadataUpdate.remoteLogSegmentId().topicIdPartition())
+                .updateRemoteLogSegmentMetadata(metadataUpdate);
+    }
+
+    private RemoteLogMetadataCache getRemoteLogMetadataCache(TopicIdPartition topicIdPartition)
+            throws RemoteResourceNotFoundException {
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No existing metadata found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache;
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       int epochForOffset,
+                                                                       long offset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition,
+                                                int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).highestOffsetForEpoch(leaderEpoch);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+
+        RemotePartitionDeleteState targetState = remotePartitionDeleteMetadata.state();
+        RemotePartitionDeleteMetadata existingMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        RemotePartitionDeleteState existingState = existingMetadata != null ? existingMetadata.state() : null;
+        if (!RemotePartitionDeleteState.isValidTransition(existingState, targetState)) {
+            throw new IllegalStateException("Current state: " + existingState + ", target state: " + targetState);
+        }
+
+        idToPartitionDeleteMetadata.put(topicIdPartition, remotePartitionDeleteMetadata);
+
+        if (targetState == RemotePartitionDeleteState.DELETE_PARTITION_FINISHED) {
+            // remove the association for the partition.
+            idToRemoteLogMetadataCache.remove(topicIdPartition);
+            idToPartitionDeleteMetadata.remove(topicIdPartition);
+        }
+    }
+
+    @Override
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition)
+            throws RemoteStorageException {
+        return getRemoteLogMetadataCache(topicIdPartition).listAllRemoteLogSegments();

Review comment:
       Should we requireNonNull for topicIdPartition here too?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no
+        // need to add for that state here.
+        return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED)

Review comment:
       Is the test for RemoteLogSegmentState.COPY_SEGMENT_FINISHED necessary since it seems that only segments with RemoteLogSegmentState.COPY_SEGMENT_FINISHED exist in offsetToId.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       When are entries in leaderEpochEntries removed?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java
##########
@@ -28,9 +27,14 @@ public RemoteStorageException(final String message) {
         super(message);
     }
 
-    public RemoteStorageException(final String message,
-                                  final Throwable cause) {
+    public RemoteStorageException(final String message, final Throwable cause) {
         super(message, cause);
     }
 
+    public RemoteStorageException() {

Review comment:
       Is this constructor needed?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogLeaderEpochState.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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 org.apache.kafka.server.log.remote.storage;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * This class represents the in-memory state of segments associated with a leader epoch. This includes the mapping of offset to
+ * segment ids and unreferenced segments which are not mapped to any offset but they exist in remote storage.
+ * <p>
+ * This is used by {@link RemoteLogMetadataCache} to track the segments for each leader epoch.
+ */
+class RemoteLogLeaderEpochState {
+
+    // It contains offset to segment ids mapping with the segment state as COPY_SEGMENT_FINISHED.
+    private final NavigableMap<Long, RemoteLogSegmentId> offsetToId = new ConcurrentSkipListMap<>();
+
+    /**
+     * It represents unreferenced segments for this leader epoch. It contains the segments still in COPY_SEGMENT_STARTED
+     * and DELETE_SEGMENT_STARTED state or these have been replaced by callers with other segments having the same
+     * start offset for the leader epoch. These will be returned by {@link RemoteLogMetadataCache#listAllRemoteLogSegments()}
+     * and {@link RemoteLogMetadataCache#listRemoteLogSegments(int leaderEpoch)} so that callers can clean them up if
+     * they still exist. These will be cleaned from the cache once they reach DELETE_SEGMENT_FINISHED state.
+     */
+    private final Set<RemoteLogSegmentId> unreferencedSegmentIds = ConcurrentHashMap.newKeySet();
+
+    // It represents the highest log offset of the segments that were updated with updateHighestLogOffset.
+    private volatile Long highestLogOffset;
+
+    /**
+     * Returns all the segments associated with this leader epoch sorted by start offset in ascending order.
+     *
+     * @param idToSegmentMetadata mapping of id to segment metadata. This will be used to get RemoteLogSegmentMetadata
+     *                            for an id to be used for sorting.
+     * @return
+     */
+    Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments(Map<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata) {
+        // Return all the segments including unreferenced metadata.
+        int size = offsetToId.size() + unreferencedSegmentIds.size();
+        if (size == 0) {
+            return Collections.emptyIterator();
+        }
+
+        ArrayList<RemoteLogSegmentMetadata> metadataList = new ArrayList<>(size);
+        for (RemoteLogSegmentId id : offsetToId.values()) {
+            metadataList.add(idToSegmentMetadata.get(id));
+        }
+
+        if (!unreferencedSegmentIds.isEmpty()) {
+            for (RemoteLogSegmentId id : unreferencedSegmentIds) {
+                metadataList.add(idToSegmentMetadata.get(id));
+            }
+
+            // sort only when unreferenced entries exist as they are already sorted in offsetToId.
+            metadataList.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        }
+
+        return metadataList.iterator();
+    }
+
+    void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                   Long leaderEpochEndOffset) {
+        // Add the segment epochs mapping as the segment is copied successfully.
+        RemoteLogSegmentId oldEntry = offsetToId.put(startOffset, remoteLogSegmentId);
+
+        // Remove the metadata from unreferenced entries as it is successfully copied and added to the offset mapping.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // Add the old entry to unreferenced entries as the mapping is removed for the old entry.
+        if (oldEntry != null) {
+            unreferencedSegmentIds.add(oldEntry);
+        }
+
+        // Update the highest offset entry for this leader epoch as we added a new mapping.
+        updateHighestLogOffset(leaderEpochEndOffset);
+    }
+
+    void handleSegmentWithDeleteSegmentStartedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                    Long leaderEpochEndOffset) {
+        // Remove the offset mappings as this segment is getting deleted.
+        offsetToId.remove(startOffset, remoteLogSegmentId);
+
+        // Add this entry to unreferenced set for the leader epoch as it is being deleted.
+        // This allows any retries of deletion as these are returned from listAllSegments and listSegments(leaderEpoch).
+        unreferencedSegmentIds.add(remoteLogSegmentId);
+
+        // Update the highest offset entry for this leader epoch. This needs to be done as a segment can reach this
+        // state without going through COPY_SEGMENT_FINISHED state.
+        updateHighestLogOffset(leaderEpochEndOffset);
+    }
+
+    void handleSegmentWithDeleteSegmentFinishedState(long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                     Long leaderEpochEndOffset) {
+        // It completely removes the tracking of this segment as it is considered as deleted.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // We do not need to update the highest offset entry with this segment as it would have already been processed
+        // as part of the earlier state of DELETE_SEGMENT_STARTED.
+    }
+
+    void handleSegmentWithCopySegmentStartedState(RemoteLogSegmentId remoteLogSegmentId) {
+        // Add this to unreferenced set of segments for the respective leader epoch.
+        unreferencedSegmentIds.add(remoteLogSegmentId);
+    }
+
+    Long highestLogOffset() {
+        return highestLogOffset;
+    }
+
+    private void updateHighestLogOffset(long offset) {

Review comment:
       updateHighestLogOffset => maybeUpdateHighestLogOffset ?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java
##########
@@ -21,14 +21,16 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Objects;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
 /**
- * It indicates the state of the remote log segment. This will be based on the action executed on this
+ * This enum indicates the state of the remote log segment. This will be based on the action executed on this
  * segment by the remote log service implementation.
  * <p>
- * It goes through the below state transitions.
+ * It goes through the below state transitions. Self transition is treated as valid. This allows updating with the
+ * same state in case of retries and failover.

Review comment:
       This is an existing issue. But is `&gt;` in line 37 expected?




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

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