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/03/11 02:40:48 UTC

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

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



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+/**
+ * This class provides an inmemory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to epoch evolution. It also keeps track of segments which are not considered to be copied to remote
+ * storage.
+ */
+public class RemoteLogMetadataCache {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new HashSet<>();
+
+    private final ConcurrentMap<Integer, NavigableMap<Long, RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    public RemoteLogMetadataCache() {

Review comment:
       This c'tor can be removed in exchange for the default generated c'tor.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+/**
+ * This class provides an inmemory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to epoch evolution. It also keeps track of segments which are not considered to be copied to remote
+ * storage.
+ */
+public class RemoteLogMetadataCache {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new HashSet<>();
+
+    private final ConcurrentMap<Integer, NavigableMap<Long, RemoteLogSegmentId>> leaderEpochToOffsetToId

Review comment:
       Looking at the implementation, it appears we maintain some rules on when a `RemoteLogSegmentId` exists in one of these data structures versus all of them. It would be useful to briefly document those rules, and mention invariants (if any). For example, when an upload is in progress it is not (yet) added to this map.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by inmemory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private final ConcurrentMap<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<TopicIdPartition, RemoteLogMetadataCache> partitionToRemoteLogMetadataCache =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        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
+                    + " but it contains state as: " + remoteLogSegmentMetadata.state());
+        }
+
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache());
+
+        remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate rlsmUpdate)
+            throws RemoteStorageException {
+        Objects.requireNonNull(rlsmUpdate, "rlsmUpdate can not be null");
+
+        // Callers should use putRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as
+        // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+        if (rlsmUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: "
+                                               + RemoteLogSegmentState.COPY_SEGMENT_STARTED);
+        }
+        log.debug("Updating remote log segment: [{}]", rlsmUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = rlsmUpdate.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No partition metadata found for : " + topicIdPartition);
+        }
+
+        remoteLogMetadataCache.updateRemoteLogSegmentMetadata(rlsmUpdate);
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       long offset,
+                                                                       int epochForOffset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No metadata found for the given partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestLogOffset(TopicIdPartition topicIdPartition,
+                                           int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No resource found for partition: " + topicIdPartition);
+        }
+
+        Long highestKey = remoteLogMetadataCache.highestLogOffset(leaderEpoch);
+        return Optional.ofNullable(highestKey);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+        idToPartitionDeleteMetadata.put(topicIdPartition, remotePartitionDeleteMetadata);
+        // there will be a trigger to receive delete partition marker and act on that to delete all the segments.

Review comment:
       Sorry I don't understand what does this comment refer to?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+/**
+ * This class provides an inmemory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to epoch evolution. It also keeps track of segments which are not considered to be copied to remote
+ * storage.
+ */
+public class RemoteLogMetadataCache {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new HashSet<>();
+
+    private final ConcurrentMap<Integer, NavigableMap<Long, RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    public RemoteLogMetadataCache() {
+    }
+
+    private void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        log.debug("Adding remote log segment metadata: [{}]", remoteLogSegmentMetadata);
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), remoteLogSegmentMetadata);
+        Map<Integer, Long> leaderEpochToOffset = remoteLogSegmentMetadata.segmentLeaderEpochs();
+        for (Map.Entry<Integer, Long> entry : leaderEpochToOffset.entrySet()) {
+            leaderEpochToOffsetToId.computeIfAbsent(entry.getKey(), k -> new ConcurrentSkipListMap<>())
+                    .put(entry.getValue(), remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+    }
+
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        NavigableMap<Long, RemoteLogSegmentId> offsetToId = leaderEpochToOffsetToId.get(leaderEpoch);
+        if (offsetToId == null || offsetToId.isEmpty()) {

Review comment:
       Can `offsetToId` be empty if it is not null?
   I understand it is right to check for emptiness here, but I was just curious to learn if it could happen in practice.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by inmemory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private final ConcurrentMap<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<TopicIdPartition, RemoteLogMetadataCache> partitionToRemoteLogMetadataCache =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        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
+                    + " but it contains state as: " + remoteLogSegmentMetadata.state());
+        }
+
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache());
+
+        remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate rlsmUpdate)
+            throws RemoteStorageException {
+        Objects.requireNonNull(rlsmUpdate, "rlsmUpdate can not be null");
+
+        // Callers should use putRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as
+        // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+        if (rlsmUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: "
+                                               + RemoteLogSegmentState.COPY_SEGMENT_STARTED);
+        }
+        log.debug("Updating remote log segment: [{}]", rlsmUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = rlsmUpdate.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No partition metadata found for : " + topicIdPartition);
+        }
+
+        remoteLogMetadataCache.updateRemoteLogSegmentMetadata(rlsmUpdate);
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       long offset,
+                                                                       int epochForOffset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No metadata found for the given partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestLogOffset(TopicIdPartition topicIdPartition,
+                                           int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No resource found for partition: " + topicIdPartition);

Review comment:
       s/resource/entry ?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+/**
+ * This class provides an inmemory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to epoch evolution. It also keeps track of segments which are not considered to be copied to remote
+ * storage.
+ */
+public class RemoteLogMetadataCache {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new HashSet<>();
+
+    private final ConcurrentMap<Integer, NavigableMap<Long, RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    public RemoteLogMetadataCache() {
+    }
+
+    private void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        log.debug("Adding remote log segment metadata: [{}]", remoteLogSegmentMetadata);
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), remoteLogSegmentMetadata);
+        Map<Integer, Long> leaderEpochToOffset = remoteLogSegmentMetadata.segmentLeaderEpochs();
+        for (Map.Entry<Integer, Long> entry : leaderEpochToOffset.entrySet()) {
+            leaderEpochToOffsetToId.computeIfAbsent(entry.getKey(), k -> new ConcurrentSkipListMap<>())
+                    .put(entry.getValue(), remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+    }
+
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        NavigableMap<Long, RemoteLogSegmentId> offsetToId = leaderEpochToOffsetToId.get(leaderEpoch);
+        if (offsetToId == null || offsetToId.isEmpty()) {
+            return Optional.empty();
+        }
+
+        // look for floor entry as the given offset may exist in this entry.
+        Map.Entry<Long, RemoteLogSegmentId> entry = offsetToId.floorEntry(offset);
+        if (entry == null) {
+            // if the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = idToSegmentMetadata.get(entry.getValue());
+        // 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 = remoteLogSegmentMetadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : remoteLogSegmentMetadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : Optional.of(remoteLogSegmentMetadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate rlsmUpdate)
+            throws RemoteResourceNotFoundException {
+        RemoteLogSegmentId remoteLogSegmentId = rlsmUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata rlsm = idToSegmentMetadata.get(remoteLogSegmentId);

Review comment:
       can `metadata` be a better variable name over `rlsm`?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by inmemory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private final ConcurrentMap<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<TopicIdPartition, RemoteLogMetadataCache> partitionToRemoteLogMetadataCache =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        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
+                    + " but it contains state as: " + remoteLogSegmentMetadata.state());
+        }
+
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache());
+
+        remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate rlsmUpdate)
+            throws RemoteStorageException {
+        Objects.requireNonNull(rlsmUpdate, "rlsmUpdate can not be null");
+
+        // Callers should use putRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as
+        // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+        if (rlsmUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) {

Review comment:
       Hmm... it seems like the only allowed state in `rlsmUpdate` is `COPY_SEGMENT_FINISHED`. Should we validate for that instead?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by inmemory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private final ConcurrentMap<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<TopicIdPartition, RemoteLogMetadataCache> partitionToRemoteLogMetadataCache =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        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
+                    + " but it contains state as: " + remoteLogSegmentMetadata.state());
+        }
+
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);

Review comment:
       Should we move this log message before L51? so that the message conveying the intent is logged first before any possible  errors.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by inmemory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private final ConcurrentMap<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<TopicIdPartition, RemoteLogMetadataCache> partitionToRemoteLogMetadataCache =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        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
+                    + " but it contains state as: " + remoteLogSegmentMetadata.state());
+        }
+
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache());
+
+        remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate rlsmUpdate)
+            throws RemoteStorageException {
+        Objects.requireNonNull(rlsmUpdate, "rlsmUpdate can not be null");
+
+        // Callers should use putRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as
+        // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+        if (rlsmUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: "
+                                               + RemoteLogSegmentState.COPY_SEGMENT_STARTED);
+        }
+        log.debug("Updating remote log segment: [{}]", rlsmUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = rlsmUpdate.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No partition metadata found for : " + topicIdPartition);
+        }
+
+        remoteLogMetadataCache.updateRemoteLogSegmentMetadata(rlsmUpdate);
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       long offset,
+                                                                       int epochForOffset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No metadata found for the given partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestLogOffset(TopicIdPartition topicIdPartition,
+                                           int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No resource found for partition: " + topicIdPartition);
+        }
+
+        Long highestKey = remoteLogMetadataCache.highestLogOffset(leaderEpoch);
+        return Optional.ofNullable(highestKey);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+        idToPartitionDeleteMetadata.put(topicIdPartition, remotePartitionDeleteMetadata);
+        // there will be a trigger to receive delete partition marker and act on that to delete all the segments.
+
+        if (remotePartitionDeleteMetadata.state() == RemotePartitionDeleteState.DELETE_PARTITION_FINISHED) {

Review comment:
       It seems like we want to add more protections here.
   
   1. If `remotePartitionDeleteMetadata.state()` is `DELETE_PARTITION_FINISHED`, then should there have been an earlier entry with `DELETE_PARTITION_STARTED` or  `DELETE_PARTITION_MARKED`?
   2. Imagine there exists an entry in `partitionToRemoteLogMetadataCache` while the partition is also being deleted. Is that a valid state, or if not should we assert against it?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by inmemory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private final ConcurrentMap<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<TopicIdPartition, RemoteLogMetadataCache> partitionToRemoteLogMetadataCache =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        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
+                    + " but it contains state as: " + remoteLogSegmentMetadata.state());
+        }
+
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache());
+
+        remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate rlsmUpdate)
+            throws RemoteStorageException {
+        Objects.requireNonNull(rlsmUpdate, "rlsmUpdate can not be null");
+
+        // Callers should use putRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as
+        // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+        if (rlsmUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: "
+                                               + RemoteLogSegmentState.COPY_SEGMENT_STARTED);
+        }
+        log.debug("Updating remote log segment: [{}]", rlsmUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = rlsmUpdate.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No partition metadata found for : " + topicIdPartition);
+        }
+
+        remoteLogMetadataCache.updateRemoteLogSegmentMetadata(rlsmUpdate);
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       long offset,
+                                                                       int epochForOffset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No metadata found for the given partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestLogOffset(TopicIdPartition topicIdPartition,
+                                           int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No resource found for partition: " + topicIdPartition);
+        }
+
+        Long highestKey = remoteLogMetadataCache.highestLogOffset(leaderEpoch);
+        return Optional.ofNullable(highestKey);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+        idToPartitionDeleteMetadata.put(topicIdPartition, remotePartitionDeleteMetadata);
+        // there will be a trigger to receive delete partition marker and act on that to delete all the segments.
+
+        if (remotePartitionDeleteMetadata.state() == RemotePartitionDeleteState.DELETE_PARTITION_FINISHED) {
+            // remove the association for the partition.
+            partitionToRemoteLogMetadataCache.remove(topicIdPartition);
+            idToPartitionDeleteMetadata.remove(topicIdPartition);
+        }
+    }
+
+    @Override
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition)
+            throws RemoteStorageException {
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No resource found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.listAllRemoteLogSegments();
+    }
+
+    @Override
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No resource found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.listRemoteLogSegments(leaderEpoch);
+    }
+
+    @Override
+    public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions,

Review comment:
       Hmm, any reason to not implement these methods? Is it that they don't serve any purpose in the in-memory implementation?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+/**
+ * This class provides an inmemory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to epoch evolution. It also keeps track of segments which are not considered to be copied to remote
+ * storage.
+ */
+public class RemoteLogMetadataCache {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new HashSet<>();
+
+    private final ConcurrentMap<Integer, NavigableMap<Long, RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    public RemoteLogMetadataCache() {
+    }
+
+    private void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        log.debug("Adding remote log segment metadata: [{}]", remoteLogSegmentMetadata);

Review comment:
       It seems like we allow for an entry already existing with the same ID to be replaced with a different entry. Would that happen in practice?




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