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



##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java
##########
@@ -87,4 +89,27 @@ public byte id() {
     public static RemoteLogSegmentState forId(byte id) {
         return STATE_TYPES.get(id);
     }
+
+    public static boolean isValidTransition(RemoteLogSegmentState srcState, 
RemoteLogSegmentState targetState) {
+        Objects.requireNonNull(targetState, "targetState can not be null");
+
+        if (srcState == null) {
+            // If the source state is null, check the target state as the 
initial state viz DELETE_PARTITION_MARKED

Review comment:
       DELETE_PARTITION_MARKED is not part of RemoteLogSegmentState.

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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<>();
+
+    // It keeps the segments which are not yet reached to 
COPY_SEGMENT_FINISHED state.
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new 
HashSet<>();
+
+    // It will have all the segments except with state as COPY_SEGMENT_STARTED.
+    private final ConcurrentMap<Integer, NavigableMap<Long, 
RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    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 metadata = 
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 = metadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : 
metadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : 
Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", 
metadataUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = 
idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment 
metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentState existingState = existingMetadata.state();
+        if (!RemoteLogSegmentState.isValidTransition(existingMetadata.state(), 
targetState)) {
+            throw new IllegalStateException("Current state: " + existingState 
+ ", target state: " + targetState);
+        }
+
+        RemoteLogSegmentMetadata updatedMetadata = 
existingMetadata.createRemoteLogSegmentWithUpdates(metadataUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedMetadata);
+        if (targetState != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedMetadata);
+        }
+
+        if (targetState == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+            // remove this entry when the state is moved to 
delete_segment_finished
+            Map<Integer, Long> leaderEpochs = 
existingMetadata.segmentLeaderEpochs();
+            for (Map.Entry<Integer, Long> entry : leaderEpochs.entrySet()) {
+                NavigableMap<Long, RemoteLogSegmentId> offsetToIds = 
leaderEpochToOffsetToId.get(entry.getKey());
+                // remove the mappings where this segment is deleted.
+                offsetToIds.values().remove(remoteLogSegmentId);

Review comment:
       It's possible that after this, there is no segment associated with a 
leader epoch. Should we remove the entry with that leader epoch from 
leaderEpochToOffsetToId?

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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<>();
+
+    // It keeps the segments which are not yet reached to 
COPY_SEGMENT_FINISHED state.
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new 
HashSet<>();
+
+    // It will have all the segments except with state as COPY_SEGMENT_STARTED.
+    private final ConcurrentMap<Integer, NavigableMap<Long, 
RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    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 metadata = 
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 = metadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : 
metadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : 
Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", 
metadataUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = 
idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment 
metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentState existingState = existingMetadata.state();
+        if (!RemoteLogSegmentState.isValidTransition(existingMetadata.state(), 
targetState)) {
+            throw new IllegalStateException("Current state: " + existingState 
+ ", target state: " + targetState);
+        }
+
+        RemoteLogSegmentMetadata updatedMetadata = 
existingMetadata.createRemoteLogSegmentWithUpdates(metadataUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedMetadata);
+        if (targetState != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedMetadata);
+        }
+
+        if (targetState == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {

Review comment:
       > This behavior was kept to be the same as local log cleanup behavior, 
in which leader epoch state is truncated only after local log is moved/deleted. 
Ideally, it is good not to consider the segments available that are being 
deleted as you said.
   
   For the local log, we first schedule the segment for async deletion and then 
take it out of leaderEpochCache. So, the equivalent of that for remote storage 
seems to require taking the segment out of leaderEpochCache once the segment 
deletion is initiated.
   

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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

Review comment:
       inmemory => in-memory

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java
##########
@@ -87,4 +89,27 @@ public byte id() {
     public static RemoteLogSegmentState forId(byte id) {
         return STATE_TYPES.get(id);
     }
+
+    public static boolean isValidTransition(RemoteLogSegmentState srcState, 
RemoteLogSegmentState targetState) {

Review comment:
       This seems to be an internal implementation and is not part of the 
public API? Ditto for the same method in RemotePartitionDeleteState.

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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<>();
+
+    // It keeps the segments which are not yet reached to 
COPY_SEGMENT_FINISHED state.
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new 
HashSet<>();
+
+    // It will have all the segments except with state as COPY_SEGMENT_STARTED.
+    private final ConcurrentMap<Integer, NavigableMap<Long, 
RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    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 metadata = 
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 = metadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : 
metadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : 
Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", 
metadataUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = 
idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment 
metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentState existingState = existingMetadata.state();
+        if (!RemoteLogSegmentState.isValidTransition(existingMetadata.state(), 
targetState)) {
+            throw new IllegalStateException("Current state: " + existingState 
+ ", target state: " + targetState);
+        }
+
+        RemoteLogSegmentMetadata updatedMetadata = 
existingMetadata.createRemoteLogSegmentWithUpdates(metadataUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedMetadata);
+        if (targetState != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedMetadata);

Review comment:
       This method updates idToSegmentMetadata, which seems redundant since 
it's done in line 107 already.

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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<>();
+
+    // It keeps the segments which are not yet reached to 
COPY_SEGMENT_FINISHED state.
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new 
HashSet<>();
+
+    // It will have all the segments except with state as COPY_SEGMENT_STARTED.
+    private final ConcurrentMap<Integer, NavigableMap<Long, 
RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    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 metadata = 
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 = metadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : 
metadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : 
Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", 
metadataUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = 
idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment 
metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentState existingState = existingMetadata.state();
+        if (!RemoteLogSegmentState.isValidTransition(existingMetadata.state(), 
targetState)) {
+            throw new IllegalStateException("Current state: " + existingState 
+ ", target state: " + targetState);
+        }
+
+        RemoteLogSegmentMetadata updatedMetadata = 
existingMetadata.createRemoteLogSegmentWithUpdates(metadataUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedMetadata);
+        if (targetState != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedMetadata);
+        }
+
+        if (targetState == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+            // remove this entry when the state is moved to 
delete_segment_finished
+            Map<Integer, Long> leaderEpochs = 
existingMetadata.segmentLeaderEpochs();
+            for (Map.Entry<Integer, Long> entry : leaderEpochs.entrySet()) {
+                NavigableMap<Long, RemoteLogSegmentId> offsetToIds = 
leaderEpochToOffsetToId.get(entry.getKey());
+                // remove the mappings where this segment is deleted.
+                offsetToIds.values().remove(remoteLogSegmentId);
+            }
+
+            // remove the segment-id mapping.
+            idToSegmentMetadata.remove(remoteLogSegmentId);
+        }
+    }
+
+    public Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments() {
+        ArrayList<RemoteLogSegmentMetadata> list = new 
ArrayList<>(idToSegmentMetadata.values());
+        list.addAll(remoteLogSegmentIdInProgress.stream().map(id -> 
idToSegmentMetadata.get(id))
+                .collect(Collectors.toList()));
+        
list.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        return list.iterator();
+    }
+
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(int 
leaderEpoch) {
+        NavigableMap<Long, RemoteLogSegmentId> map = 
leaderEpochToOffsetToId.get(leaderEpoch);
+        return map != null ? map.values().stream().map(id -> 
idToSegmentMetadata.get(id)).iterator()
+                           : Collections.emptyIterator();
+    }
+
+    public Optional<Long> highestLogOffset(int leaderEpoch) {

Review comment:
       highestLogOffset => highestOffsetForEpoch?

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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<>();
+
+    // It keeps the segments which are not yet reached to 
COPY_SEGMENT_FINISHED state.
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new 
HashSet<>();
+
+    // It will have all the segments except with state as COPY_SEGMENT_STARTED.
+    private final ConcurrentMap<Integer, NavigableMap<Long, 
RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    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 metadata = 
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 = metadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : 
metadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : 
Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", 
metadataUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = 
idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment 
metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentState existingState = existingMetadata.state();
+        if (!RemoteLogSegmentState.isValidTransition(existingMetadata.state(), 
targetState)) {
+            throw new IllegalStateException("Current state: " + existingState 
+ ", target state: " + targetState);
+        }
+
+        RemoteLogSegmentMetadata updatedMetadata = 
existingMetadata.createRemoteLogSegmentWithUpdates(metadataUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedMetadata);
+        if (targetState != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedMetadata);
+        }
+
+        if (targetState == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+            // remove this entry when the state is moved to 
delete_segment_finished
+            Map<Integer, Long> leaderEpochs = 
existingMetadata.segmentLeaderEpochs();
+            for (Map.Entry<Integer, Long> entry : leaderEpochs.entrySet()) {
+                NavigableMap<Long, RemoteLogSegmentId> offsetToIds = 
leaderEpochToOffsetToId.get(entry.getKey());
+                // remove the mappings where this segment is deleted.
+                offsetToIds.values().remove(remoteLogSegmentId);
+            }
+
+            // remove the segment-id mapping.
+            idToSegmentMetadata.remove(remoteLogSegmentId);
+        }
+    }
+
+    public Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments() {
+        ArrayList<RemoteLogSegmentMetadata> list = new 
ArrayList<>(idToSegmentMetadata.values());
+        list.addAll(remoteLogSegmentIdInProgress.stream().map(id -> 
idToSegmentMetadata.get(id))
+                .collect(Collectors.toList()));
+        
list.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        return list.iterator();
+    }
+
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(int 
leaderEpoch) {
+        NavigableMap<Long, RemoteLogSegmentId> map = 
leaderEpochToOffsetToId.get(leaderEpoch);
+        return map != null ? map.values().stream().map(id -> 
idToSegmentMetadata.get(id)).iterator()
+                           : Collections.emptyIterator();
+    }
+
+    public Optional<Long> highestLogOffset(int leaderEpoch) {
+        NavigableMap<Long, RemoteLogSegmentId> offsetToSegmentId = 
leaderEpochToOffsetToId.get(leaderEpoch);
+        if (offsetToSegmentId == null) {
+            return Optional.empty();
+        }
+
+        long max = 0L;
+        for (RemoteLogSegmentId id : offsetToSegmentId.values()) {

Review comment:
       It's kind of inefficient to have to iterate through the whole segment 
list. Could we make leaderEpochToOffsetToId an ordered map and then do 
highEntry on that?

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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<>();
+
+    // It keeps the segments which are not yet reached to 
COPY_SEGMENT_FINISHED state.
+    private final Set<RemoteLogSegmentId> remoteLogSegmentIdInProgress = new 
HashSet<>();
+
+    // It will have all the segments except with state as COPY_SEGMENT_STARTED.
+    private final ConcurrentMap<Integer, NavigableMap<Long, 
RemoteLogSegmentId>> leaderEpochToOffsetToId
+            = new ConcurrentHashMap<>();
+
+    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 metadata = 
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 = metadata.segmentLeaderEpochs()
+                .higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : 
metadata.endOffset();
+
+        // seek offset should be <= epoch's end offset.
+        return (offset > epochEndOffset) ? Optional.empty() : 
Optional.of(metadata);
+    }
+
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteResourceNotFoundException {
+        log.debug("Updating remote log segment metadata: [{}]", 
metadataUpdate);
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        RemoteLogSegmentMetadata existingMetadata = 
idToSegmentMetadata.get(remoteLogSegmentId);
+        if (existingMetadata == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment 
metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        RemoteLogSegmentState existingState = existingMetadata.state();
+        if (!RemoteLogSegmentState.isValidTransition(existingMetadata.state(), 
targetState)) {
+            throw new IllegalStateException("Current state: " + existingState 
+ ", target state: " + targetState);
+        }
+
+        RemoteLogSegmentMetadata updatedMetadata = 
existingMetadata.createRemoteLogSegmentWithUpdates(metadataUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedMetadata);
+        if (targetState != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedMetadata);
+        }
+
+        if (targetState == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+            // remove this entry when the state is moved to 
delete_segment_finished
+            Map<Integer, Long> leaderEpochs = 
existingMetadata.segmentLeaderEpochs();
+            for (Map.Entry<Integer, Long> entry : leaderEpochs.entrySet()) {
+                NavigableMap<Long, RemoteLogSegmentId> offsetToIds = 
leaderEpochToOffsetToId.get(entry.getKey());
+                // remove the mappings where this segment is deleted.
+                offsetToIds.values().remove(remoteLogSegmentId);
+            }
+
+            // remove the segment-id mapping.
+            idToSegmentMetadata.remove(remoteLogSegmentId);
+        }
+    }
+
+    public Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments() {
+        ArrayList<RemoteLogSegmentMetadata> list = new 
ArrayList<>(idToSegmentMetadata.values());
+        list.addAll(remoteLogSegmentIdInProgress.stream().map(id -> 
idToSegmentMetadata.get(id))

Review comment:
       > There may be few segments with state as COPY_SEGMENT_STARTED and they 
will be part of remoteLogSegmentIdInProgress only but not idToSegmentMetadata. 
That is why we need to add them to the list.
   
   Hmm, it seems that we add the in-progress segment to idToSegmentMetadata in 
addToInProgress? It would be useful to add a comment for idToSegmentMetadata.

##########
File path: 
remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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 {
+        log.debug("Adding remote log segment : [{}]", 
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
+                    + " but it contains state as: " + 
remoteLogSegmentMetadata.state());
+        }
+
+        RemoteLogSegmentId remoteLogSegmentId = 
remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        RemoteLogMetadataCache remoteLogMetadataCache = 
partitionToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> 
new RemoteLogMetadataCache());
+
+        remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate 
metadataUpdate)
+            throws RemoteStorageException {
+        log.debug("Updating remote log segment: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be 
null");
+
+        RemoteLogSegmentState targetState = metadataUpdate.state();
+        // Callers should use putRemoteLogSegmentMetadata to add 
RemoteLogSegmentMetadata with state as
+        // RemoteLogSegmentState.COPY_SEGMENT_STARTED.
+        if (targetState == RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            throw new IllegalArgumentException("Given remoteLogSegmentMetadata 
should not have the state as: "
+                                               + 
RemoteLogSegmentState.COPY_SEGMENT_STARTED);
+        }
+
+        RemoteLogSegmentId remoteLogSegmentId = 
metadataUpdate.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = 
remoteLogSegmentId.topicIdPartition();
+        RemoteLogMetadataCache remoteLogMetadataCache = 
partitionToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No metadata found for 
partition: " + topicIdPartition);
+        }
+
+        remoteLogMetadataCache.updateRemoteLogSegmentMetadata(metadataUpdate);
+    }
+
+    @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 metadata found for 
partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache.highestLogOffset(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.
+            partitionToRemoteLogMetadataCache.remove(topicIdPartition);
+            idToPartitionDeleteMetadata.remove(topicIdPartition);
+        }
+    }
+
+    @Override
+    public Iterator<RemoteLogSegmentMetadata> 
listRemoteLogSegments(TopicIdPartition topicIdPartition)

Review comment:
       It seems that there is a semantic difference between this method and the 
next one. While this one exposes all segments (including in progress ones), the 
latter only exposes segments that are completed. It would be useful to document 
this clearly in the public API.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to