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/02/26 08:48:01 UTC

[GitHub] [kafka] satishd opened a new pull request #10218: KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

satishd opened a new pull request #10218:
URL: https://github.com/apache/kafka/pull/10218


   KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.
   
   Added inmemory implementation for RemoteStorageManager and RemoteLogMetadataManager. A major part of inmemory RLMM will be used in the default RLMM implementation which will be based on topic storage. These will be used in unit tests for tiered storage.
   Added a few tests for both the implementations.
   
   This is part of tiered storage implementation, KIP-405.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592437366



##########
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:
       1 -> Added more assertions. 
   2 -> is a valid case.
   




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-797234253


   thanks @junrao @kowshik for the review comments, addressed with the latest commits/comments.
   


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-815062947


   @junrao This PR is on top of https://github.com/apache/kafka/pull/10489 . So, https://github.com/apache/kafka/pull/10489 be merged before merging this PR. 


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607551710



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache = new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        idToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache())
+                .addCopyInProgressSegment(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteStorageException {
+        log.debug("Updating remote log segment: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        getRemoteLogMetadataCache(metadataUpdate.remoteLogSegmentId().topicIdPartition())
+                .updateRemoteLogSegmentMetadata(metadataUpdate);
+    }
+
+    private RemoteLogMetadataCache getRemoteLogMetadataCache(TopicIdPartition topicIdPartition)
+            throws RemoteResourceNotFoundException {
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No existing metadata found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache;
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       int epochForOffset,
+                                                                       long offset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition,
+                                                int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).highestOffsetForEpoch(leaderEpoch);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+
+        RemotePartitionDeleteState targetState = remotePartitionDeleteMetadata.state();
+        RemotePartitionDeleteMetadata existingMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        RemotePartitionDeleteState existingState = existingMetadata != null ? existingMetadata.state() : null;
+        if (!RemotePartitionDeleteState.isValidTransition(existingState, targetState)) {
+            throw new IllegalStateException("Current state: " + existingState + ", target state: " + targetState);

Review comment:
       It prints null. I may be missing something here. What needs to be handled here?




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607959239



##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,48 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     * @return

Review comment:
       nit: remove empty `@return`

##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,48 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     * @return
+     */
+    public static <T> boolean sameElementsWithOrder(Iterator<T> iterator1,
+                                                    Iterator<T> iterator2) {
+        while (iterator1.hasNext()) {
+            if (!iterator2.hasNext()) {
+                return false;
+            }
+
+            Object elem1 = iterator1.next();
+            Object elem2 = iterator2.next();
+            if (!Objects.equals(elem1, elem2)) {
+                return false;
+            }
+        }
+
+        return !iterator2.hasNext();
+    }
+
+    /**
+     * Returns true if both the iterators have same set of elements irrespective of order and duplicates.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     * @return

Review comment:
       nit: remove empty `@return`

##########
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) {

Review comment:
       Same comment as before: https://github.com/apache/kafka/pull/10218/files#r598982742.
   Can srcState be null in practice? If not, this can be defined as an instance method.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicates it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicates it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicates it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <pre>
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * |                                 | COPY_SEGMENT_STARTED | COPY_SEGMENT_FINISHED  | DELETE_SEGMENT_STARTED  | DELETE_SEGMENT_STARTED  |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | remoteLogSegmentMetadata        |        No            |           Yes          |          No             |           No            |
+ * | (int leaderEpoch, long offset)  |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listRemoteLogSegments           |        Yes           |           Yes          |          Yes            |           No            |
+ * | (int leaderEpoch)               |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listAllRemoteLogSegments()      |        Yes           |           Yes          |          Yes            |           No            |
+ * |                                 |                      |                        |                         |                         |
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * </pre>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       Should we call this map as `idToLeaderEpochState` or `idToEpochState` similar to the naming for the other map?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592173915



##########
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:
       Other states include `COPY_SEGMENT_FINISHED`, `DELETE_SEGMENT_STARTED`, and `DELETE_SEGMENT_FINISHED`.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-814275837


   @junrao I renamed `remote-storage` module to `storage` module as you suggested in the commit [009d7fa](https://github.com/apache/kafka/pull/10218/commits/009d7fa6c861c671f3b2aa85c05fdd5ff467e195).


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591585302



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

Review comment:
       The plan was to use the related classes in the default RLMM implementation and move any class which is relevant only for tests to test dir later. I am +1 to have this as s separate module. I will update with those changes. 
   




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591595247



##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", rlsmUpdate);
+            // remove this entry when the state is moved to delete_segment_finished
+            Map<Integer, Long> leaderEpochs = rlsm.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. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r595954157



##########
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 is not really an internal implementation but it validates the state transition and it is the same for any implementation.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592074962



##########
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:
       "No <s>resource</s> metadata found for partition: "?




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591588474



##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);

Review comment:
       This behavior was kept to be the same as local log cleanup behavior, in which leader epoch 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.  




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609387061



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCacheTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * This class covers basic unit tests for {@link InmemoryRemoteLogMetadataManager}. InmemoryRemoteLogMetadataManager is
+ * used only in integration tests but not in production code. It mostly uses {@link RemoteLogMetadataCache} and it has
+ * broad test coverage with {@link RemoteLogMetadataCacheTest}.
+ */
+public class InmemoryRemoteLogMetadataManagerTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testFetchSegments() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and this segment should not be available.
+        Map<Integer, Long> segmentLeaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 101L, 200L, -1L, BROKER_ID_0,
+                time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segmentMetadata);
+
+        // Search should not return the above segment.
+        Assertions.assertFalse(rlmm.remoteLogSegmentMetadata(TP0, 0, 150).isPresent());
+
+        // 2.Move that segment to COPY_SEGMENT_FINISHED state and this segment should be available.
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+        RemoteLogSegmentMetadata expectedSegmentMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> segmentMetadataForOffset150 = rlmm.remoteLogSegmentMetadata(TP0, 0, 150);
+        Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadataForOffset150);
+    }
+
+    @Test
+    public void testRemotePartitionDeletion() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // Create remote log segment metadata and add them to RLMM.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segmentLeaderEpochs = new HashMap<>();
+        segmentLeaderEpochs.put(0, 0L);
+        segmentLeaderEpochs.put(1, 20L);
+        segmentLeaderEpochs.put(2, 50L);
+        segmentLeaderEpochs.put(3, 80L);
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segmentMetadata);
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(
+                segmentId, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+
+        RemoteLogSegmentMetadata expectedSegMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate);
+
+        // Check that the seg exists in RLMM
+        Optional<RemoteLogSegmentMetadata> segMetadataForOffset30Epoch1 = rlmm.remoteLogSegmentMetadata(TP0, 1, 30L);
+        Assertions.assertEquals(Optional.of(expectedSegMetadata), segMetadataForOffset30Epoch1);
+
+        // Mark the partition for deletion. RLMM should clear all its internal state for that partition.

Review comment:
       Updated.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596083656



##########
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:
       Right, we do not need to add `remoteLogSegmentIdInProgress ` here. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r595954157



##########
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 is not really an internal implementation but it validates the transition and it is the same for any implementation.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610464252



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

Review comment:
       Addressed in the above [comment](https://github.com/apache/kafka/pull/10218#discussion_r610462859). 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-816736436


   Thanks @junrao @kowshik for the comments. Addressed them with the [d12c77a](https://github.com/apache/kafka/pull/10218/commits/d12c77a8f91d15cc9dd8b4c3f664f38c51c033b7).


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r595954157



##########
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 is not really an internal implementation but the method which validates the transition and it is the same for any implementation.




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609998576



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicates it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicates it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicates it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <pre>
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * |                                 | COPY_SEGMENT_STARTED | COPY_SEGMENT_FINISHED  | DELETE_SEGMENT_STARTED  | DELETE_SEGMENT_STARTED  |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | remoteLogSegmentMetadata        |        No            |           Yes          |          No             |           No            |
+ * | (int leaderEpoch, long offset)  |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listRemoteLogSegments           |        Yes           |           Yes          |          Yes            |           No            |
+ * | (int leaderEpoch)               |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listAllRemoteLogSegments()      |        Yes           |           Yes          |          Yes            |           No            |
+ * |                                 |                      |                        |                         |                         |
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * </pre>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       `leaderEpochToState` sounds good.




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610491754



##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,46 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     */
+    public static <T> boolean sameElementsWithOrder(Iterator<T> iterator1,

Review comment:
       Sounds good




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591603442



##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", rlsmUpdate);
+            // remove this entry when the state is moved to delete_segment_finished
+            Map<Integer, Long> leaderEpochs = rlsm.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 Long highestLogOffset(int leaderEpoch) {

Review comment:
       No, it is not highestSegmentStartOffset but it is the highest log offset for the given leader epoch.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609352593



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segment0LeaderEpochs = new HashMap<>();
+        segment0LeaderEpochs.put(0, 0L);
+        segment0LeaderEpochs.put(1, 20L);
+        segment0LeaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs);
+        cache.addCopyInProgressSegment(segment0Metadata);
+
+        // We should not get this as the segment is still getting copied and it is not yet considered successful until
+        // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate(
+                segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segment0Update);
+        RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update);
+
+        // segment 1
+        // 101 - 200
+        // no changes in leadership with in this segment
+        // leader epochs (2, 101)
+        Map<Integer, Long> segment1LeaderEpochs = Collections.singletonMap(2, 101L);
+        RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 2
+        // 201 - 300
+        // moved to epoch 3 in between
+        // leader epochs (2, 201), (3, 240)
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(2, 201L);
+        segment2LeaderEpochs.put(3, 240L);
+        RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 3
+        // 250 - 400
+        // leader epochs (3, 250), (4, 370)
+        Map<Integer, Long> segment3LeaderEpochs = new HashMap<>();
+        segment3LeaderEpochs.put(3, 250L);
+        segment3LeaderEpochs.put(4, 370L);
+        RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        // Four segments are added with different boundaries and leader epochs.
+        // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset)  for different
+        // epochs and offsets
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        HashMap<EpochOffset, RemoteLogSegmentMetadata> expectedEpochOffsetToSegmentMetadata = new HashMap<>();
+        // Existing metadata entries.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata);
+
+        // Non existing metadata entries.
+        // Search for offset 110, epoch 1, and it should not exist.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null);
+        // Search for non existing offset 401, epoch 4.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null);
+        // Search for non existing epoch 5.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null);
+
+        for (Map.Entry<EpochOffset, RemoteLogSegmentMetadata> entry : expectedEpochOffsetToSegmentMetadata.entrySet()) {
+            EpochOffset epochOffset = entry.getKey();
+            Optional<RemoteLogSegmentMetadata> segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset);
+            RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue();
+            log.info("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata,

Review comment:
       We may have this as debug level by default. It will be helpful to see for which `EpochOffset` the test is failed. 




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



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

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610809710



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

Review comment:
       Typo epty




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



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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607559905



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no
+        // need to add for that state here.
+        return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED)

Review comment:
       good point, this check is no more needed. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610462859



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

Review comment:
       It already replaces the existing entry [here](https://github.com/apache/kafka/pull/10218/files#diff-3724bb53d7ab4bc5a6ec4e1ab4c91c47bf90e4166d881f7706e2adc1848a5d16R299). 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610517070



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

Review comment:
       Good point, I will add a check for that. 




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



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

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609895193



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

Review comment:
       > One way to do that is to clear the entry when the respective RemoteLogLeaderEpochState is empty. That means all the segments reached DELETE_SEGMENT_FINISHED state.
   > This is not currently addressed. I plan to look into it when we integrate these APIs with RemoteLogManager by exploring other options too.
   
   Could we add a TODO comment here so that we don't forget about it?

##########
File path: settings.gradle
##########
@@ -29,6 +29,7 @@ include 'clients',
     'log4j-appender',
     'metadata',
     'raft',
+    'storage',

Review comment:
       This is redundant.

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+public class InmemoryRemoteStorageManagerTest {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManagerTest.class);
+
+    private static final TopicPartition TP = new TopicPartition("foo", 1);
+    private static final File DIR = TestUtils.tempDirectory("inmem-rsm-");
+    private static final Random RANDOM = new Random();
+
+    @Test
+    public void testCopyLogSegment() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        LogSegmentData logSegmentData = createLogSegmentData();
+        // Copy all the segment data.
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check that the segment data exists in in-memory RSM.
+        boolean containsSegment = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForSegment(segmentMetadata));
+        Assertions.assertTrue(containsSegment);
+
+        // Check that the indexes exist in in-memory RSM.
+        for (RemoteStorageManager.IndexType indexType : RemoteStorageManager.IndexType.values()) {
+            boolean containsIndex = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForIndex(segmentMetadata, indexType));
+            Assertions.assertTrue(containsIndex);
+        }
+    }
+
+    private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() {
+        TopicIdPartition topicPartition = new TopicIdPartition(Uuid.randomUuid(), TP);
+        RemoteLogSegmentId id = new RemoteLogSegmentId(topicPartition, Uuid.randomUuid());
+        return new RemoteLogSegmentMetadata(id, 100L, 200L, System.currentTimeMillis(), 0,
+                System.currentTimeMillis(), 100, Collections.singletonMap(1, 100L));
+    }
+
+    @Test
+    public void testFetchLogSegmentIndexes() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        int segSize = 100;
+        LogSegmentData logSegmentData = createLogSegmentData(segSize);
+
+        // Copy the segment
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check segment data exists for the copied segment.
+        try (InputStream segmentStream = rsm.fetchLogSegment(segmentMetadata, 0)) {
+            checkContentSame(segmentStream, logSegmentData.logSegment());
+        }
+
+        HashMap<RemoteStorageManager.IndexType, Path> expectedIndexToPaths = new HashMap<>();
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.OFFSET, logSegmentData.offsetIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TIMESTAMP, logSegmentData.timeIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TRANSACTION, logSegmentData.txnIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, logSegmentData.producerSnapshotIndex());
+
+        // Check all segment indexes exist for the copied segment.
+        for (Map.Entry<RemoteStorageManager.IndexType, Path> entry : expectedIndexToPaths.entrySet()) {
+            RemoteStorageManager.IndexType indexType = entry.getKey();
+            Path indexPath = entry.getValue();
+            log.info("Fetching index type: {}, indexPath: {}", indexType, indexPath);

Review comment:
       Could we move this to debug level then?

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

Review comment:
       Sounds good. Could you make the change in the PR?




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596032415



##########
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:
       I will update the javadoc of the APIs to make this clear.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596247752



##########
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
+            // Wanted to keep this logic simple here by taking null for srcState, instead of creating one more state like
+            // COPY_SEGMENT_NOT_STARTED and have the null check by caller and pass that state.
+            return targetState == COPY_SEGMENT_STARTED;
+        } else if (srcState == targetState) {

Review comment:
       1 -> imho, this validation method should be part of the state enum and it can be used by any implementation including default RLMM. 
   
   2 -> I would have preferred the suggested approach if there are many complex transitions but the transitions here are few and simple. 




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r595493561



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

Review comment:
       I have the same suggestions from `RemoteLogSegmentState` for this as well. Please refer to this comment: https://github.com/apache/kafka/pull/10218#discussion_r595492577




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596083656



##########
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:
       Right, we do not need to add `remoteLogSegmentIdInProgress ` here. 




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610305248



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

Review comment:
       Is this method expected to be idempotent?
   This comment is related to my other comment: https://github.com/apache/kafka/pull/10218#discussion_r610298531.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610454559



##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,46 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     */
+    public static <T> boolean sameElementsWithOrder(Iterator<T> iterator1,

Review comment:
       IMHO, existing code looks easy to read/comprehend, and no multiple calls to hasNext().
   How about the below code after removing inline variables in the existing code?
   
   ```
   while (iterator1.hasNext()) {
       if (!iterator2.hasNext()) {
           return false;
       }
   
       if (!Objects.equals(iterator1.next(), iterator2.next())) {
           return false;
       }
   }
   
   return !iterator2.hasNext();
   ```




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609369185



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.

Review comment:
       Sure, I will add the doc. They are currently not thread safe. But we want to address them when we integrate these APIs. 

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

Review comment:
       After thinking through this more, we need to update this only when the segment reaches COPY_SEGMENT_FINISHED. This is effectively used to find out up to which offset the segments are already copied. I will remove the call here and keep the call only in handleSegmentWithCopySegmentFinishedState. WDYT?




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r595492577



##########
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
+            // Wanted to keep this logic simple here by taking null for srcState, instead of creating one more state like
+            // COPY_SEGMENT_NOT_STARTED and have the null check by caller and pass that state.
+            return targetState == COPY_SEGMENT_STARTED;
+        } else if (srcState == targetState) {

Review comment:
       1. Will it be useful to place the implementation of this validation in a separate module, so that it can be reused with `RLMMWithTopicStorage` in the future?
   2. Suggestion from the standpoint of code readability: Would it make sense to replace the `if-else` logic by looking up from a `Map< RemoteLogSegmentState, Set< RemoteLogSegmentState>>` where key is the source state and value is a set of allowed target states?
   
   

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

Review comment:
       We may want to think more about the locking semantics for this class and `RemoteLogMetadataCache`. 
   Are we sure there would _not_ be use cases where we need to serialize mutations across the individually thread-safe attributes? If the answer is no, then using a fine-grained `Object` lock makes more sense because we can use it to guard critical sections.
   
   Should we evaluate this upfront?
   
   cc @junrao 

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

Review comment:
       I have the same suggestions from `RemoteLogSegmentState` for this as well. Please refer to this comment:

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

Review comment:
       In the comment:
   s/putRemoteLogSegmentMetadata/addRemoteLogSegmentMetadata
   
   

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

Review comment:
       Really minor comment/discussion: Any reason to call this prefixed with `add` as `addRemoteLogSegmentMetadata` vs calling the deletion one prefixed with `put` as `putRemotePartitionDeleteMetadata` i.e. instead can these 2 methods both start with the same prefix either `add` or `put`?




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r598964030



##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java
##########
@@ -21,14 +21,16 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Objects;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
 /**
  * It indicates the state of the remote log segment. This will be based on the action executed on this

Review comment:
       You can drop `It` and start with `Indicates the state...`.

##########
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) {

Review comment:
       Can this be checked inside `RemoteLogMetadataCache.addToInProgress()` instead of here?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),
+                    Files.readAllBytes(logSegmentData.logSegment().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Offset),
+                    Files.readAllBytes(logSegmentData.offsetIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Timestamp),
+                    Files.readAllBytes(logSegmentData.timeIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Transaction),
+                    Files.readAllBytes(logSegmentData.txnIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LeaderEpoch),
+                    logSegmentData.leaderEpochIndex().array());
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.ProducerSnapshot),
+                    Files.readAllBytes(logSegmentData.producerSnapshotIndex().toPath()));
+        } catch (IOException e) {
+            throw new RemoteStorageException(e.getMessage(), e);

Review comment:
       We could add a c'tor overload to [RemoteStorageException](https://github.com/apache/kafka/blob/0d9a95a7d0ab06aecc4480901707e29dd2a3147e/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java) that takes a `Throwable` as argument, it would the need to pass 2 args here.

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),
+                    Files.readAllBytes(logSegmentData.logSegment().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Offset),
+                    Files.readAllBytes(logSegmentData.offsetIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Timestamp),
+                    Files.readAllBytes(logSegmentData.timeIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Transaction),
+                    Files.readAllBytes(logSegmentData.txnIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LeaderEpoch),
+                    logSegmentData.leaderEpochIndex().array());
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.ProducerSnapshot),
+                    Files.readAllBytes(logSegmentData.producerSnapshotIndex().toPath()));
+        } catch (IOException e) {
+            throw new RemoteStorageException(e.getMessage(), e);
+        }
+        log.debug("copied log segment and indexes for : {} successfully.", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition)
+            throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] for [{}]", startPosition, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        return fetchLogSegment(remoteLogSegmentMetadata, startPosition, Integer.MAX_VALUE);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition,
+                                       int endPosition) throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] and end position: [{}] for segment [{}]",
+                startPosition, endPosition, remoteLogSegmentMetadata);
+
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        if (startPosition < 0 || endPosition < 0) {
+            throw new IllegalArgumentException("Given start position or end position must not be negative.");
+        }
+
+        if (endPosition < startPosition) {
+            throw new IllegalArgumentException("end position must be greater than start position");
+        }
+
+        String key = generateKeyForSegment(remoteLogSegmentMetadata);
+        byte[] segment = keyToLogData.get(key);
+
+        if (segment == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment found with start offset:"
+                                                      + remoteLogSegmentMetadata.startOffset() + " and id: "
+                                                      + remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+
+        if (startPosition >= segment.length) {
+            throw new IllegalArgumentException("start position: " + startPosition
+                                               + " must be less than the length of the segment: " + segment.length);
+        }
+
+        // check for boundaries like given end position is more than the length, length should never be more than the
+        // existing segment size.
+        int length = Math.min(segment.length - 1, endPosition) - startPosition + 1;

Review comment:
       Hmm, do we need to explicitly check if `endPosition` < `segment.length`? 

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java
##########
@@ -83,4 +85,25 @@ public static RemotePartitionDeleteState forId(byte id) {
         return STATE_TYPES.get(id);
     }
 
+    public static boolean isValidTransition(RemotePartitionDeleteState srcState,
+                                            RemotePartitionDeleteState 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
+            // Wanted to keep this logic simple here by taking null for srcState, instead of creating one more state like
+            // DELETE_PARTITION_NOT_MARKED and have the null check by caller and pass that state.

Review comment:
       IMHO, we can simplify this to say:
   
   ```
   // If the source state is null, check the target state as the initial state viz DELETE_PARTITION_MARKED.
   // This ensures simplicity as we don't have to define an additional state type to represent the initial state.
   ```

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Add segments with different states and check cache.remoteLogSegmentMetadata(int leaderEpoch, long offset)
+        // cache.listRemoteLogSegments(int leaderEpoch), and cache.listAllRemoteLogSegments().
+
+        // =============================================================================================================
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg0leaderEpochs = Collections.singletonMap(0, 0L);
+        RemoteLogSegmentId seg0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segCopyInProgress = new RemoteLogSegmentMetadata(seg0Id, 0L, 50L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg0leaderEpochs);
+        cache.addToInProgress(segCopyInProgress);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> seg0s0e0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(seg0s0e0.isPresent());
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment, it will be empty.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        // But cache.listRemoteLogSegments() should contain the above segment.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Collections.singletonList(segCopyInProgress));
+
+        // =============================================================================================================
+        // 2.Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg1leaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId seg1Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg1 = new RemoteLogSegmentMetadata(seg1Id, 101L, 200L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg1leaderEpochs);
+        cache.addToInProgress(seg1);
+        RemoteLogSegmentMetadataUpdate seg1Update = new RemoteLogSegmentMetadataUpdate(seg1Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg1Update);
+        RemoteLogSegmentMetadata segCopyFinished = seg1.createRemoteLogSegmentWithUpdates(seg1Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg1S150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(seg1.createRemoteLogSegmentWithUpdates(seg1Update), seg1S150.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Collections.singletonList(segCopyFinished));
+        // But cache.listRemoteLogSegments() should contain both the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Arrays.asList(segCopyInProgress, segCopyFinished));
+
+        // =============================================================================================================
+        // 3.Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg2leaderEpochs = Collections.singletonMap(0, 201L);
+        RemoteLogSegmentId seg2Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg2 = new RemoteLogSegmentMetadata(seg2Id, 201L, 300L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg2leaderEpochs);
+        cache.addToInProgress(seg2);
+        RemoteLogSegmentMetadataUpdate seg2Update = new RemoteLogSegmentMetadataUpdate(seg2Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg2Update);
+        RemoteLogSegmentMetadata segDeleteStarted = seg2.createRemoteLogSegmentWithUpdates(seg2Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg2S250 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertEquals(seg2.createRemoteLogSegmentWithUpdates(seg2Update), seg2S250.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should contain all the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+
+        // =============================================================================================================
+        // 4.Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg3leaderEpochs = Collections.singletonMap(0, 301L);
+        RemoteLogSegmentId seg3Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg3 = new RemoteLogSegmentMetadata(seg3Id, 301L, 400L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg3leaderEpochs);
+        cache.addToInProgress(seg3);
+        RemoteLogSegmentMetadataUpdate seg3Update1 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg3Update1);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg3S350 = cache.remoteLogSegmentMetadata(0, 350);
+        Assertions.assertEquals(seg3.createRemoteLogSegmentWithUpdates(seg3Update1), seg3S350.orElse(null));
+
+        RemoteLogSegmentMetadataUpdate seg3Update2 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg3Update2);
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should not contain both the segments as it should have been removed.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+    }
+
+    private void checkContainsAll(Iterator<RemoteLogSegmentMetadata> allSegments,

Review comment:
       The implementation compromises on the ordering, since it converts the iterator to a set. Is that intentional?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Add segments with different states and check cache.remoteLogSegmentMetadata(int leaderEpoch, long offset)
+        // cache.listRemoteLogSegments(int leaderEpoch), and cache.listAllRemoteLogSegments().
+
+        // =============================================================================================================
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg0leaderEpochs = Collections.singletonMap(0, 0L);
+        RemoteLogSegmentId seg0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segCopyInProgress = new RemoteLogSegmentMetadata(seg0Id, 0L, 50L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg0leaderEpochs);
+        cache.addToInProgress(segCopyInProgress);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> seg0s0e0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(seg0s0e0.isPresent());
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment, it will be empty.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        // But cache.listRemoteLogSegments() should contain the above segment.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Collections.singletonList(segCopyInProgress));
+
+        // =============================================================================================================
+        // 2.Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg1leaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId seg1Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg1 = new RemoteLogSegmentMetadata(seg1Id, 101L, 200L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg1leaderEpochs);
+        cache.addToInProgress(seg1);
+        RemoteLogSegmentMetadataUpdate seg1Update = new RemoteLogSegmentMetadataUpdate(seg1Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg1Update);
+        RemoteLogSegmentMetadata segCopyFinished = seg1.createRemoteLogSegmentWithUpdates(seg1Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg1S150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(seg1.createRemoteLogSegmentWithUpdates(seg1Update), seg1S150.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Collections.singletonList(segCopyFinished));
+        // But cache.listRemoteLogSegments() should contain both the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Arrays.asList(segCopyInProgress, segCopyFinished));
+
+        // =============================================================================================================
+        // 3.Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg2leaderEpochs = Collections.singletonMap(0, 201L);
+        RemoteLogSegmentId seg2Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg2 = new RemoteLogSegmentMetadata(seg2Id, 201L, 300L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg2leaderEpochs);
+        cache.addToInProgress(seg2);
+        RemoteLogSegmentMetadataUpdate seg2Update = new RemoteLogSegmentMetadataUpdate(seg2Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg2Update);
+        RemoteLogSegmentMetadata segDeleteStarted = seg2.createRemoteLogSegmentWithUpdates(seg2Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg2S250 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertEquals(seg2.createRemoteLogSegmentWithUpdates(seg2Update), seg2S250.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should contain all the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+
+        // =============================================================================================================
+        // 4.Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg3leaderEpochs = Collections.singletonMap(0, 301L);
+        RemoteLogSegmentId seg3Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg3 = new RemoteLogSegmentMetadata(seg3Id, 301L, 400L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg3leaderEpochs);
+        cache.addToInProgress(seg3);
+        RemoteLogSegmentMetadataUpdate seg3Update1 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg3Update1);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg3S350 = cache.remoteLogSegmentMetadata(0, 350);
+        Assertions.assertEquals(seg3.createRemoteLogSegmentWithUpdates(seg3Update1), seg3S350.orElse(null));

Review comment:
       Could we assert just before this line that `seg3S350` is not empty? this will simplify the `seg3S350.orElse(null)` argument to `seg3S350.get()`.
   
   (same comment applies for other places in this test)

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

Review comment:
       This is defined to be not thread safe unlike the other maps. Is there any reason?

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

Review comment:
       Could we call this `idToRemoteLogMetadataCache` to align with the naming of the other attribute thats called `idToPartitionDeleteMetadata` ?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+public class InmemoryRemoteLogMetadataManagerTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testRLMMFetchSegment() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+        int brokerId = 0;
+        // Create remote log segment metadata and add them to RLMM.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> seg0leaderEpochs = new HashMap<>();
+        seg0leaderEpochs.put(0, 0L);
+        seg0leaderEpochs.put(1, 20L);
+        seg0leaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segIdFooTp0s0e100 = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segMetFooTp0s0e100 = new RemoteLogSegmentMetadata(segIdFooTp0s0e100, 0L, 100L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg0leaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segMetFooTp0s0e100);
+
+        // wWe should not get this as the segment is still gettign copied and it is not yet considered successful until

Review comment:
       Typos:
   1. s/wWe/We
   2. s/gettign/getting

##########
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) {

Review comment:
       Similar to above comment, why not check this inside `remoteLogMetadataCache.updateRemoteLogSegmentMetadata()`?

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

Review comment:
       Can we add a 1-line doc for this similar to other attributes below?

##########
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);
+        }
+

Review comment:
       In this method, we allow for existing entries in `idToSegmentMetadata` to be replaced, even if the state of the existing and new entries are the same. Is that intentional?

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

Review comment:
       Hmm, the entry for `existingMetadata` gets overwritten in the call to `addRemoteLogSegmentMetadata` in L110. Should we be accounting for the same here?

##########
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()) {
+            RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(id);
+            Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+            // If there is an higher entry than the given leader epoch, end-offset of the given leader epoch is,
+            // (next leader epoch's start-offset) -1.
+            long nextVal = nextEntry != null ? nextEntry.getValue() - 1 : metadata.endOffset();
+            max = Math.max(max, nextVal);
+        }
+
+        return Optional.of(max);
+    }
+
+    /**
+     * This will be added to copy_in_progress metadata list. This will be removed from that list once it is moved to the
+     * next state which can be COPY_SEGMENT_FINISHED or DELETE_SEGMENT_STARTED.
+     *
+     * @param remoteLogSegmentMetadata
+     */
+    public void addToInProgress(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {

Review comment:
       Before we insert into the map/set, we should check if the provided `remoteLogSegmentMetadata.state()` is `COPY_SEGMENT_STARTED`.

##########
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) {

Review comment:
       It seems to me that `srcState` is never null in practice. Where does this check come into play in practice?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+public class InmemoryRemoteLogMetadataManagerTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testRLMMFetchSegment() throws Exception {

Review comment:
       Can we improve the local variable names? for example `segIdFooTp0s0e100`, `segMetFooTp0s0e100` etc. is not easy to read. We can use simpler names.

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {

Review comment:
       Could we add test(s) for `highestLogOffset` API?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Add segments with different states and check cache.remoteLogSegmentMetadata(int leaderEpoch, long offset)
+        // cache.listRemoteLogSegments(int leaderEpoch), and cache.listAllRemoteLogSegments().
+
+        // =============================================================================================================
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg0leaderEpochs = Collections.singletonMap(0, 0L);
+        RemoteLogSegmentId seg0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segCopyInProgress = new RemoteLogSegmentMetadata(seg0Id, 0L, 50L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg0leaderEpochs);
+        cache.addToInProgress(segCopyInProgress);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> seg0s0e0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(seg0s0e0.isPresent());
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment, it will be empty.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        // But cache.listRemoteLogSegments() should contain the above segment.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Collections.singletonList(segCopyInProgress));
+
+        // =============================================================================================================
+        // 2.Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg1leaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId seg1Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg1 = new RemoteLogSegmentMetadata(seg1Id, 101L, 200L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg1leaderEpochs);
+        cache.addToInProgress(seg1);
+        RemoteLogSegmentMetadataUpdate seg1Update = new RemoteLogSegmentMetadataUpdate(seg1Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg1Update);
+        RemoteLogSegmentMetadata segCopyFinished = seg1.createRemoteLogSegmentWithUpdates(seg1Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg1S150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(seg1.createRemoteLogSegmentWithUpdates(seg1Update), seg1S150.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Collections.singletonList(segCopyFinished));
+        // But cache.listRemoteLogSegments() should contain both the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Arrays.asList(segCopyInProgress, segCopyFinished));
+
+        // =============================================================================================================
+        // 3.Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg2leaderEpochs = Collections.singletonMap(0, 201L);
+        RemoteLogSegmentId seg2Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg2 = new RemoteLogSegmentMetadata(seg2Id, 201L, 300L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg2leaderEpochs);
+        cache.addToInProgress(seg2);
+        RemoteLogSegmentMetadataUpdate seg2Update = new RemoteLogSegmentMetadataUpdate(seg2Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg2Update);
+        RemoteLogSegmentMetadata segDeleteStarted = seg2.createRemoteLogSegmentWithUpdates(seg2Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg2S250 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertEquals(seg2.createRemoteLogSegmentWithUpdates(seg2Update), seg2S250.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should contain all the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+
+        // =============================================================================================================
+        // 4.Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg3leaderEpochs = Collections.singletonMap(0, 301L);
+        RemoteLogSegmentId seg3Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg3 = new RemoteLogSegmentMetadata(seg3Id, 301L, 400L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg3leaderEpochs);
+        cache.addToInProgress(seg3);
+        RemoteLogSegmentMetadataUpdate seg3Update1 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);

Review comment:
       Should we alter the other arguments too, for example `BROKER_ID` and `eventTimestamp`? It appears that we expect `RemoteLogMetadataCache` to [apply all of the provided updates](https://github.com/apache/kafka/blob/0d9a95a7d0ab06aecc4480901707e29dd2a3147e/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java#L240-L242), and this may include the other fields as well.

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),

Review comment:
       As per the interface we [mandate](https://github.com/apache/kafka/blob/0d9a95a7d0ab06aecc4480901707e29dd2a3147e/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java#L76) the caller to ensure unique ID, but is it useful to add a guard that disallows replacing existing values?)

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {

Review comment:
       Can we remove this c'tor in exchange for the default generated c'tor?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {

Review comment:
       This particular test checks a number of things together in one test. Instead, could sections (1) to (4) from below each be defined as a separate unit test? Especially since each section seems to operate on a different segment, so it seems logically independent.

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),
+                    Files.readAllBytes(logSegmentData.logSegment().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Offset),
+                    Files.readAllBytes(logSegmentData.offsetIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Timestamp),
+                    Files.readAllBytes(logSegmentData.timeIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Transaction),
+                    Files.readAllBytes(logSegmentData.txnIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LeaderEpoch),
+                    logSegmentData.leaderEpochIndex().array());
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.ProducerSnapshot),
+                    Files.readAllBytes(logSegmentData.producerSnapshotIndex().toPath()));
+        } catch (IOException e) {
+            throw new RemoteStorageException(e.getMessage(), e);
+        }
+        log.debug("copied log segment and indexes for : {} successfully.", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition)
+            throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] for [{}]", startPosition, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        return fetchLogSegment(remoteLogSegmentMetadata, startPosition, Integer.MAX_VALUE);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition,
+                                       int endPosition) throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] and end position: [{}] for segment [{}]",
+                startPosition, endPosition, remoteLogSegmentMetadata);
+
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        if (startPosition < 0 || endPosition < 0) {
+            throw new IllegalArgumentException("Given start position or end position must not be negative.");
+        }
+
+        if (endPosition < startPosition) {
+            throw new IllegalArgumentException("end position must be greater than start position");
+        }
+
+        String key = generateKeyForSegment(remoteLogSegmentMetadata);
+        byte[] segment = keyToLogData.get(key);
+
+        if (segment == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment found with start offset:"
+                                                      + remoteLogSegmentMetadata.startOffset() + " and id: "
+                                                      + remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+
+        if (startPosition >= segment.length) {
+            throw new IllegalArgumentException("start position: " + startPosition
+                                               + " must be less than the length of the segment: " + segment.length);
+        }
+
+        // check for boundaries like given end position is more than the length, length should never be more than the
+        // existing segment size.
+        int length = Math.min(segment.length - 1, endPosition) - startPosition + 1;
+        log.debug("Length of the segment to be sent: [{}], for segment: [{}]", length, remoteLogSegmentMetadata);
+
+        return new ByteArrayInputStream(segment, startPosition, length);
+    }
+
+    @Override
+    public InputStream fetchIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                  IndexType indexType) throws RemoteStorageException {
+        log.debug("Received fetch request for index type: [{}], segment [{}]", indexType, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(indexType, "indexType can not be null");
+
+        String key = generateKeyForIndex(remoteLogSegmentMetadata, indexType);
+        byte[] index = keyToLogData.get(key);
+        if (index == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment index found with start offset:"
+                                                      + remoteLogSegmentMetadata.startOffset() + " and id: "
+                                                      + remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+
+        return new ByteArrayInputStream(index);
+    }
+
+    @Override
+    public void deleteLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException {
+        log.info("Deleting log segment for: [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        String segmentKey = generateKeyForSegment(remoteLogSegmentMetadata);
+        keyToLogData.remove(segmentKey);
+        for (IndexType indexType : IndexType.values()) {
+            String key = generateKeyForIndex(remoteLogSegmentMetadata, indexType);
+            keyToLogData.remove(key);
+        }
+        log.info("Deleted log segment successfully for: [{}]", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void close() throws IOException {
+        keyToLogData = Collections.emptyMap();
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {

Review comment:
       Is this intentionally left empty?

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),
+                    Files.readAllBytes(logSegmentData.logSegment().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Offset),
+                    Files.readAllBytes(logSegmentData.offsetIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Timestamp),
+                    Files.readAllBytes(logSegmentData.timeIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Transaction),
+                    Files.readAllBytes(logSegmentData.txnIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LeaderEpoch),
+                    logSegmentData.leaderEpochIndex().array());
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.ProducerSnapshot),
+                    Files.readAllBytes(logSegmentData.producerSnapshotIndex().toPath()));
+        } catch (IOException e) {
+            throw new RemoteStorageException(e.getMessage(), e);
+        }
+        log.debug("copied log segment and indexes for : {} successfully.", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition)
+            throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] for [{}]", startPosition, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        return fetchLogSegment(remoteLogSegmentMetadata, startPosition, Integer.MAX_VALUE);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition,
+                                       int endPosition) throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] and end position: [{}] for segment [{}]",
+                startPosition, endPosition, remoteLogSegmentMetadata);
+
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        if (startPosition < 0 || endPosition < 0) {
+            throw new IllegalArgumentException("Given start position or end position must not be negative.");
+        }
+
+        if (endPosition < startPosition) {
+            throw new IllegalArgumentException("end position must be greater than start position");

Review comment:
       Probably better to say `...must be greater than or equal to...` ?




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r608820288



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicates it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicates it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicates it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <pre>
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * |                                 | COPY_SEGMENT_STARTED | COPY_SEGMENT_FINISHED  | DELETE_SEGMENT_STARTED  | DELETE_SEGMENT_STARTED  |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | remoteLogSegmentMetadata        |        No            |           Yes          |          No             |           No            |
+ * | (int leaderEpoch, long offset)  |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listRemoteLogSegments           |        Yes           |           Yes          |          Yes            |           No            |
+ * | (int leaderEpoch)               |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listAllRemoteLogSegments()      |        Yes           |           Yes          |          Yes            |           No            |
+ * |                                 |                      |                        |                         |                         |
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * </pre>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       The key is not really `id` but `epoch` no. What about `remoteLogLeaderEpochStateEntries` or `leaderEpochToState` or any other better name? 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606253220



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {

Review comment:
       I thought earlier about having different methods, but it checks `listAllSegments/listSegment(leaderEpoch)` APIs that return earlier segments. But I will have a separate test for that and extract as suggested. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606251255



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Add segments with different states and check cache.remoteLogSegmentMetadata(int leaderEpoch, long offset)
+        // cache.listRemoteLogSegments(int leaderEpoch), and cache.listAllRemoteLogSegments().
+
+        // =============================================================================================================
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg0leaderEpochs = Collections.singletonMap(0, 0L);
+        RemoteLogSegmentId seg0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segCopyInProgress = new RemoteLogSegmentMetadata(seg0Id, 0L, 50L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg0leaderEpochs);
+        cache.addToInProgress(segCopyInProgress);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> seg0s0e0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(seg0s0e0.isPresent());
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment, it will be empty.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        // But cache.listRemoteLogSegments() should contain the above segment.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Collections.singletonList(segCopyInProgress));
+
+        // =============================================================================================================
+        // 2.Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg1leaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId seg1Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg1 = new RemoteLogSegmentMetadata(seg1Id, 101L, 200L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg1leaderEpochs);
+        cache.addToInProgress(seg1);
+        RemoteLogSegmentMetadataUpdate seg1Update = new RemoteLogSegmentMetadataUpdate(seg1Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg1Update);
+        RemoteLogSegmentMetadata segCopyFinished = seg1.createRemoteLogSegmentWithUpdates(seg1Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg1S150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(seg1.createRemoteLogSegmentWithUpdates(seg1Update), seg1S150.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Collections.singletonList(segCopyFinished));
+        // But cache.listRemoteLogSegments() should contain both the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Arrays.asList(segCopyInProgress, segCopyFinished));
+
+        // =============================================================================================================
+        // 3.Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg2leaderEpochs = Collections.singletonMap(0, 201L);
+        RemoteLogSegmentId seg2Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg2 = new RemoteLogSegmentMetadata(seg2Id, 201L, 300L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg2leaderEpochs);
+        cache.addToInProgress(seg2);
+        RemoteLogSegmentMetadataUpdate seg2Update = new RemoteLogSegmentMetadataUpdate(seg2Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg2Update);
+        RemoteLogSegmentMetadata segDeleteStarted = seg2.createRemoteLogSegmentWithUpdates(seg2Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg2S250 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertEquals(seg2.createRemoteLogSegmentWithUpdates(seg2Update), seg2S250.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should contain all the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+
+        // =============================================================================================================
+        // 4.Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg3leaderEpochs = Collections.singletonMap(0, 301L);
+        RemoteLogSegmentId seg3Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg3 = new RemoteLogSegmentMetadata(seg3Id, 301L, 400L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg3leaderEpochs);
+        cache.addToInProgress(seg3);
+        RemoteLogSegmentMetadataUpdate seg3Update1 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg3Update1);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg3S350 = cache.remoteLogSegmentMetadata(0, 350);
+        Assertions.assertEquals(seg3.createRemoteLogSegmentWithUpdates(seg3Update1), seg3S350.orElse(null));
+
+        RemoteLogSegmentMetadataUpdate seg3Update2 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg3Update2);
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should not contain both the segments as it should have been removed.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+    }
+
+    private void checkContainsAll(Iterator<RemoteLogSegmentMetadata> allSegments,

Review comment:
       Good catch. This is addressed in the latest commit.




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r611267145



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

Review comment:
       typo: The title of the last column should be `DELETE_SEGMENT_FINISHED`.




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



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

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606343187



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be

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

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {

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

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

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

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogLeaderEpochState.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * This class represents the in-memory state of segments associated with a leader epoch. This includes the mapping of offset to
+ * segment ids and unreferenced segments which are not mapped to any offset but they exist in remote storage.
+ * <p>
+ * This is used by {@link RemoteLogMetadataCache} to track the segments for each leader epoch.
+ */
+class RemoteLogLeaderEpochState {
+
+    // It contains offset to segment ids mapping with the segment state as COPY_SEGMENT_FINISHED.
+    private final NavigableMap<Long, RemoteLogSegmentId> offsetToId = new ConcurrentSkipListMap<>();
+
+    /**
+     * It represents unreferenced segments for this leader epoch. It contains the segments still in COPY_SEGMENT_STARTED
+     * and DELETE_SEGMENT_STARTED state or these have been replaced by callers with other segments having the same
+     * start offset for the leader epoch. These will be returned by {@link RemoteLogMetadataCache#listAllRemoteLogSegments()}
+     * and {@link RemoteLogMetadataCache#listRemoteLogSegments(int leaderEpoch)} so that callers can clean them up if
+     * they still exist. These will be cleaned from the cache once they reach DELETE_SEGMENT_FINISHED state.
+     */
+    private final Set<RemoteLogSegmentId> unreferencedSegmentIds = ConcurrentHashMap.newKeySet();
+
+    // It represents the highest log offset of the segments that were updated with updateHighestLogOffset.
+    private volatile Long highestLogOffset;
+
+    /**
+     * Returns all the segments associated with this leader epoch sorted by start offset in ascending order.
+     *
+     * @param idToSegmentMetadata mapping of id to segment metadata. This will be used to get RemoteLogSegmentMetadata
+     *                            for an id to be used for sorting.
+     * @return
+     */
+    Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments(Map<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata) {
+        // Return all the segments including unreferenced metadata.
+        int size = offsetToId.size() + unreferencedSegmentIds.size();
+        if (size == 0) {
+            return Collections.emptyIterator();
+        }
+
+        ArrayList<RemoteLogSegmentMetadata> metadataList = new ArrayList<>(size);
+        for (RemoteLogSegmentId id : offsetToId.values()) {
+            metadataList.add(idToSegmentMetadata.get(id));
+        }
+
+        if (!unreferencedSegmentIds.isEmpty()) {
+            for (RemoteLogSegmentId id : unreferencedSegmentIds) {
+                metadataList.add(idToSegmentMetadata.get(id));
+            }
+
+            // sort only when unreferenced entries exist as they are already sorted in offsetToId.
+            metadataList.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        }
+
+        return metadataList.iterator();
+    }
+
+    void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                   Long leaderEpochEndOffset) {
+        // Add the segment epochs mapping as the segment is copied successfully.
+        RemoteLogSegmentId oldEntry = offsetToId.put(startOffset, remoteLogSegmentId);
+
+        // Remove the metadata from unreferenced entries as it is successfully copied and added to the offset mapping.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // Add the old entry to unreferenced entries as the mapping is removed for the old entry.
+        if (oldEntry != null) {
+            unreferencedSegmentIds.add(oldEntry);

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

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>

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

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no

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

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

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

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

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

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache = new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        idToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache())
+                .addCopyInProgressSegment(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteStorageException {
+        log.debug("Updating remote log segment: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        getRemoteLogMetadataCache(metadataUpdate.remoteLogSegmentId().topicIdPartition())
+                .updateRemoteLogSegmentMetadata(metadataUpdate);
+    }
+
+    private RemoteLogMetadataCache getRemoteLogMetadataCache(TopicIdPartition topicIdPartition)
+            throws RemoteResourceNotFoundException {
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No existing metadata found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache;
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       int epochForOffset,
+                                                                       long offset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition,
+                                                int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).highestOffsetForEpoch(leaderEpoch);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+
+        RemotePartitionDeleteState targetState = remotePartitionDeleteMetadata.state();
+        RemotePartitionDeleteMetadata existingMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        RemotePartitionDeleteState existingState = existingMetadata != null ? existingMetadata.state() : null;
+        if (!RemotePartitionDeleteState.isValidTransition(existingState, targetState)) {
+            throw new IllegalStateException("Current state: " + existingState + ", target state: " + targetState);

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

##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.
+ */
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class);
+
+    private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
+            new ConcurrentHashMap<>();
+
+    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache = new ConcurrentHashMap<>();
+
+    @Override
+    public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+
+        idToRemoteLogMetadataCache
+                .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache())
+                .addCopyInProgressSegment(remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate)
+            throws RemoteStorageException {
+        log.debug("Updating remote log segment: [{}]", metadataUpdate);
+        Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null");
+
+        getRemoteLogMetadataCache(metadataUpdate.remoteLogSegmentId().topicIdPartition())
+                .updateRemoteLogSegmentMetadata(metadataUpdate);
+    }
+
+    private RemoteLogMetadataCache getRemoteLogMetadataCache(TopicIdPartition topicIdPartition)
+            throws RemoteResourceNotFoundException {
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache == null) {
+            throw new RemoteResourceNotFoundException("No existing metadata found for partition: " + topicIdPartition);
+        }
+
+        return remoteLogMetadataCache;
+    }
+
+    @Override
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                       int epochForOffset,
+                                                                       long offset)
+            throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).remoteLogSegmentMetadata(epochForOffset, offset);
+    }
+
+    @Override
+    public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition,
+                                                int leaderEpoch) throws RemoteStorageException {
+        Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null");
+
+        return getRemoteLogMetadataCache(topicIdPartition).highestOffsetForEpoch(leaderEpoch);
+    }
+
+    @Override
+    public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException {
+        log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata);
+        Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null");
+
+        TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition();
+
+        RemotePartitionDeleteState targetState = remotePartitionDeleteMetadata.state();
+        RemotePartitionDeleteMetadata existingMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        RemotePartitionDeleteState existingState = existingMetadata != null ? existingMetadata.state() : null;
+        if (!RemotePartitionDeleteState.isValidTransition(existingState, targetState)) {
+            throw new IllegalStateException("Current state: " + existingState + ", target state: " + targetState);
+        }
+
+        idToPartitionDeleteMetadata.put(topicIdPartition, remotePartitionDeleteMetadata);
+
+        if (targetState == RemotePartitionDeleteState.DELETE_PARTITION_FINISHED) {
+            // remove the association for the partition.
+            idToRemoteLogMetadataCache.remove(topicIdPartition);
+            idToPartitionDeleteMetadata.remove(topicIdPartition);
+        }
+    }
+
+    @Override
+    public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition)
+            throws RemoteStorageException {
+        return getRemoteLogMetadataCache(topicIdPartition).listAllRemoteLogSegments();

Review comment:
       Should we requireNonNull for topicIdPartition here too?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no
+        // need to add for that state here.
+        return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED)

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

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       When are entries in leaderEpochEntries removed?

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

Review comment:
       Is this constructor needed?

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogLeaderEpochState.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * This class represents the in-memory state of segments associated with a leader epoch. This includes the mapping of offset to
+ * segment ids and unreferenced segments which are not mapped to any offset but they exist in remote storage.
+ * <p>
+ * This is used by {@link RemoteLogMetadataCache} to track the segments for each leader epoch.
+ */
+class RemoteLogLeaderEpochState {
+
+    // It contains offset to segment ids mapping with the segment state as COPY_SEGMENT_FINISHED.
+    private final NavigableMap<Long, RemoteLogSegmentId> offsetToId = new ConcurrentSkipListMap<>();
+
+    /**
+     * It represents unreferenced segments for this leader epoch. It contains the segments still in COPY_SEGMENT_STARTED
+     * and DELETE_SEGMENT_STARTED state or these have been replaced by callers with other segments having the same
+     * start offset for the leader epoch. These will be returned by {@link RemoteLogMetadataCache#listAllRemoteLogSegments()}
+     * and {@link RemoteLogMetadataCache#listRemoteLogSegments(int leaderEpoch)} so that callers can clean them up if
+     * they still exist. These will be cleaned from the cache once they reach DELETE_SEGMENT_FINISHED state.
+     */
+    private final Set<RemoteLogSegmentId> unreferencedSegmentIds = ConcurrentHashMap.newKeySet();
+
+    // It represents the highest log offset of the segments that were updated with updateHighestLogOffset.
+    private volatile Long highestLogOffset;
+
+    /**
+     * Returns all the segments associated with this leader epoch sorted by start offset in ascending order.
+     *
+     * @param idToSegmentMetadata mapping of id to segment metadata. This will be used to get RemoteLogSegmentMetadata
+     *                            for an id to be used for sorting.
+     * @return
+     */
+    Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments(Map<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata) {
+        // Return all the segments including unreferenced metadata.
+        int size = offsetToId.size() + unreferencedSegmentIds.size();
+        if (size == 0) {
+            return Collections.emptyIterator();
+        }
+
+        ArrayList<RemoteLogSegmentMetadata> metadataList = new ArrayList<>(size);
+        for (RemoteLogSegmentId id : offsetToId.values()) {
+            metadataList.add(idToSegmentMetadata.get(id));
+        }
+
+        if (!unreferencedSegmentIds.isEmpty()) {
+            for (RemoteLogSegmentId id : unreferencedSegmentIds) {
+                metadataList.add(idToSegmentMetadata.get(id));
+            }
+
+            // sort only when unreferenced entries exist as they are already sorted in offsetToId.
+            metadataList.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        }
+
+        return metadataList.iterator();
+    }
+
+    void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                   Long leaderEpochEndOffset) {
+        // Add the segment epochs mapping as the segment is copied successfully.
+        RemoteLogSegmentId oldEntry = offsetToId.put(startOffset, remoteLogSegmentId);
+
+        // Remove the metadata from unreferenced entries as it is successfully copied and added to the offset mapping.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // Add the old entry to unreferenced entries as the mapping is removed for the old entry.
+        if (oldEntry != null) {
+            unreferencedSegmentIds.add(oldEntry);
+        }
+
+        // Update the highest offset entry for this leader epoch as we added a new mapping.
+        updateHighestLogOffset(leaderEpochEndOffset);
+    }
+
+    void handleSegmentWithDeleteSegmentStartedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                    Long leaderEpochEndOffset) {
+        // Remove the offset mappings as this segment is getting deleted.
+        offsetToId.remove(startOffset, remoteLogSegmentId);
+
+        // Add this entry to unreferenced set for the leader epoch as it is being deleted.
+        // This allows any retries of deletion as these are returned from listAllSegments and listSegments(leaderEpoch).
+        unreferencedSegmentIds.add(remoteLogSegmentId);
+
+        // Update the highest offset entry for this leader epoch. This needs to be done as a segment can reach this
+        // state without going through COPY_SEGMENT_FINISHED state.
+        updateHighestLogOffset(leaderEpochEndOffset);
+    }
+
+    void handleSegmentWithDeleteSegmentFinishedState(long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                     Long leaderEpochEndOffset) {
+        // It completely removes the tracking of this segment as it is considered as deleted.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // We do not need to update the highest offset entry with this segment as it would have already been processed
+        // as part of the earlier state of DELETE_SEGMENT_STARTED.
+    }
+
+    void handleSegmentWithCopySegmentStartedState(RemoteLogSegmentId remoteLogSegmentId) {
+        // Add this to unreferenced set of segments for the respective leader epoch.
+        unreferencedSegmentIds.add(remoteLogSegmentId);
+    }
+
+    Long highestLogOffset() {
+        return highestLogOffset;
+    }
+
+    private void updateHighestLogOffset(long offset) {

Review comment:
       updateHighestLogOffset => maybeUpdateHighestLogOffset ?

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

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




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [kafka] satishd edited a comment on pull request #10218: KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

Posted by GitBox <gi...@apache.org>.
satishd edited a comment on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-812572844


   @junrao @kowshik thanks for your review and comments. In the latest commit, addressed the review comments. I have also refactored the code with better abstractions. 


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592071729



##########
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:
       Right, it does not make sense for inmemory implementation. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592071729



##########
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:
       Right, it is not applicable for inmemory implementation.




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



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

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609073411



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store.

Review comment:
       It would be useful to add a comment on whether the methods in this class are thread-safe or not.

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

Review comment:
       It seems that it's inconsistent that we update highest log offset here but not in handleSegmentWithCopySegmentStartedState(). 
   
   Could we comment on whether highestLogOffset reflects the segments that have reached  COPY_SEGMENT_FINISHED or not?

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segment0LeaderEpochs = new HashMap<>();
+        segment0LeaderEpochs.put(0, 0L);
+        segment0LeaderEpochs.put(1, 20L);
+        segment0LeaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs);
+        cache.addCopyInProgressSegment(segment0Metadata);
+
+        // We should not get this as the segment is still getting copied and it is not yet considered successful until
+        // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate(
+                segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segment0Update);
+        RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update);
+
+        // segment 1
+        // 101 - 200
+        // no changes in leadership with in this segment
+        // leader epochs (2, 101)
+        Map<Integer, Long> segment1LeaderEpochs = Collections.singletonMap(2, 101L);
+        RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 2
+        // 201 - 300
+        // moved to epoch 3 in between
+        // leader epochs (2, 201), (3, 240)
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(2, 201L);
+        segment2LeaderEpochs.put(3, 240L);
+        RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 3
+        // 250 - 400
+        // leader epochs (3, 250), (4, 370)
+        Map<Integer, Long> segment3LeaderEpochs = new HashMap<>();
+        segment3LeaderEpochs.put(3, 250L);
+        segment3LeaderEpochs.put(4, 370L);
+        RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        // Four segments are added with different boundaries and leader epochs.
+        // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset)  for different
+        // epochs and offsets
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        HashMap<EpochOffset, RemoteLogSegmentMetadata> expectedEpochOffsetToSegmentMetadata = new HashMap<>();
+        // Existing metadata entries.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata);
+
+        // Non existing metadata entries.
+        // Search for offset 110, epoch 1, and it should not exist.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null);
+        // Search for non existing offset 401, epoch 4.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null);
+        // Search for non existing epoch 5.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null);
+
+        for (Map.Entry<EpochOffset, RemoteLogSegmentMetadata> entry : expectedEpochOffsetToSegmentMetadata.entrySet()) {
+            EpochOffset epochOffset = entry.getKey();
+            Optional<RemoteLogSegmentMetadata> segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset);
+            RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue();
+            log.info("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata,
+                    expectedSegmentMetadata);
+            if (expectedSegmentMetadata != null) {
+                Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadata);
+            } else {
+                Assertions.assertFalse(segmentMetadata.isPresent());
+            }
+        }
+
+        // Update segment with state as DELETE_SEGMENT_STARTED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        // Update segment with state as DELETE_SEGMENT_FINISHED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        //  Search for cache.highestLogOffset(leaderEpoch) for all the leader epochs
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        Map<Integer, Long> expectedEpochToHighestOffset = new HashMap<>();
+        expectedEpochToHighestOffset.put(0, 19L);

Review comment:
       It's kind of weird that the segment with epoch 0 is already deleted and yet we still expect the highest offset for epoch 0 to be returned.

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segment0LeaderEpochs = new HashMap<>();
+        segment0LeaderEpochs.put(0, 0L);
+        segment0LeaderEpochs.put(1, 20L);
+        segment0LeaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs);
+        cache.addCopyInProgressSegment(segment0Metadata);
+
+        // We should not get this as the segment is still getting copied and it is not yet considered successful until
+        // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate(
+                segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segment0Update);
+        RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update);
+
+        // segment 1
+        // 101 - 200
+        // no changes in leadership with in this segment
+        // leader epochs (2, 101)
+        Map<Integer, Long> segment1LeaderEpochs = Collections.singletonMap(2, 101L);
+        RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 2
+        // 201 - 300
+        // moved to epoch 3 in between
+        // leader epochs (2, 201), (3, 240)
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(2, 201L);
+        segment2LeaderEpochs.put(3, 240L);
+        RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 3
+        // 250 - 400
+        // leader epochs (3, 250), (4, 370)
+        Map<Integer, Long> segment3LeaderEpochs = new HashMap<>();
+        segment3LeaderEpochs.put(3, 250L);
+        segment3LeaderEpochs.put(4, 370L);
+        RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        // Four segments are added with different boundaries and leader epochs.
+        // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset)  for different
+        // epochs and offsets
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        HashMap<EpochOffset, RemoteLogSegmentMetadata> expectedEpochOffsetToSegmentMetadata = new HashMap<>();
+        // Existing metadata entries.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata);
+
+        // Non existing metadata entries.
+        // Search for offset 110, epoch 1, and it should not exist.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null);
+        // Search for non existing offset 401, epoch 4.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null);
+        // Search for non existing epoch 5.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null);
+
+        for (Map.Entry<EpochOffset, RemoteLogSegmentMetadata> entry : expectedEpochOffsetToSegmentMetadata.entrySet()) {
+            EpochOffset epochOffset = entry.getKey();
+            Optional<RemoteLogSegmentMetadata> segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset);
+            RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue();
+            log.info("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata,
+                    expectedSegmentMetadata);
+            if (expectedSegmentMetadata != null) {
+                Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadata);
+            } else {
+                Assertions.assertFalse(segmentMetadata.isPresent());
+            }
+        }
+
+        // Update segment with state as DELETE_SEGMENT_STARTED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        // Update segment with state as DELETE_SEGMENT_FINISHED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        //  Search for cache.highestLogOffset(leaderEpoch) for all the leader epochs
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        Map<Integer, Long> expectedEpochToHighestOffset = new HashMap<>();
+        expectedEpochToHighestOffset.put(0, 19L);
+        expectedEpochToHighestOffset.put(1, 79L);
+        expectedEpochToHighestOffset.put(2, 239L);
+        expectedEpochToHighestOffset.put(3, 369L);
+        expectedEpochToHighestOffset.put(4, 400L);
+
+        for (Map.Entry<Integer, Long> entry : expectedEpochToHighestOffset.entrySet()) {
+            Integer epoch = entry.getKey();
+            Long expectedOffset = entry.getValue();
+            Optional<Long> offset = cache.highestOffsetForEpoch(epoch);
+            log.info("Fetching highest offset for epoch: {} , returned: {} , expected: {}", epoch, offset, expectedOffset);
+            Assertions.assertEquals(Optional.of(expectedOffset), offset);
+        }
+
+        // Search for non existing leader epoch
+        Optional<Long> highestOffsetForEpoch5 = cache.highestOffsetForEpoch(5);
+        Assertions.assertFalse(highestOffsetForEpoch5.isPresent());
+    }
+
+    private RemoteLogSegmentMetadata createSegmentUpdateWithState(RemoteLogMetadataCache cache,
+                                                                  Map<Integer, Long> segmentLeaderEpochs,
+                                                                  long startOffset,
+                                                                  long endOffset,
+                                                                  RemoteLogSegmentState state)
+            throws RemoteResourceNotFoundException {
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, startOffset, endOffset, -1L,
+                BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        cache.addCopyInProgressSegment(segmentMetadata);
+
+        RemoteLogSegmentMetadataUpdate segMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId,
+                time.milliseconds(), state, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segMetadataUpdate);
+
+        return segmentMetadata.createWithUpdates(segMetadataUpdate);
+    }
+
+    @Test
+    public void testCacheSegmentWithCopySegmentStartedState() {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 50L, -1L, BROKER_ID_0,
+                time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        cache.addCopyInProgressSegment(segmentMetadata);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> segMetadataForOffset0Epoch0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(segMetadataForOffset0Epoch0.isPresent());
+
+        // cache.listRemoteLogSegments APIs should contain the above segment.
+        checkListSegments(cache, 0, segmentMetadata);
+    }
+
+    @Test
+    public void testCacheSegmentWithCopySegmentFinishedState() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 101L),
+                101L, 200L, RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> segMetadataForOffset150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(Optional.of(segmentMetadata), segMetadataForOffset150);
+
+        // cache.listRemoteLogSegments should contain the above segments.
+        checkListSegments(cache, 0, segmentMetadata);
+    }
+
+    @Test
+    public void testCacheSegmentWithDeleteSegmentStartedState() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 201L),
+                201L, 300L, RemoteLogSegmentState.DELETE_SEGMENT_STARTED);
+
+        // Search should not return the above segment as their leader epoch state is cleared.
+        Optional<RemoteLogSegmentMetadata> segmentMetadataForOffset250Epoch0 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertFalse(segmentMetadataForOffset250Epoch0.isPresent());
+
+        checkListSegments(cache, 0, segmentMetadata);
+    }
+
+    @Test
+    public void testCacheSegmentsWithDeleteSegmentFinishedState() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 301L),
+                301L, 400L, RemoteLogSegmentState.DELETE_SEGMENT_STARTED);
+
+        // Search should not return the above segment as their leader epoch state is cleared.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 350).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+
+        // listRemoteLogSegments(0) and listRemoteLogSegments() should not contain the above segment.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        Assertions.assertFalse(cache.listAllRemoteLogSegments().hasNext());
+    }
+
+    @Test
+    public void testCacheListSegments() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a few segments and add them to the cache.
+        RemoteLogSegmentMetadata segment0 = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 0L), 0, 100,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+        RemoteLogSegmentMetadata segment1 = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 101L), 101, 200,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(0, 201L);
+        segment2LeaderEpochs.put(1, 301L);
+        RemoteLogSegmentMetadata segment2 = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201, 400,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // listRemoteLogSegments(0) and listAllRemoteLogSegments() should contain all the above segments.
+        List<RemoteLogSegmentMetadata> expectedSegmentsForEpoch0 = Arrays.asList(segment0, segment1, segment2);
+        Assertions.assertTrue(TestUtils.sameElementsWithOrder(cache.listRemoteLogSegments(0),
+                expectedSegmentsForEpoch0.iterator()));
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(cache.listAllRemoteLogSegments(),
+                expectedSegmentsForEpoch0.iterator()));
+
+        // listRemoteLogSegments(0) should contain only segment2.

Review comment:
       listRemoteLogSegments(0) => listRemoteLogSegments(1)

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+public class InmemoryRemoteStorageManagerTest {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManagerTest.class);
+
+    private static final TopicPartition TP = new TopicPartition("foo", 1);
+    private static final File DIR = TestUtils.tempDirectory("inmem-rsm-");
+    private static final Random RANDOM = new Random();
+
+    @Test
+    public void testCopyLogSegment() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        LogSegmentData logSegmentData = createLogSegmentData();
+        // Copy all the segment data.
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check that the segment data exists in in-memory RSM.
+        boolean containsSegment = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForSegment(segmentMetadata));
+        Assertions.assertTrue(containsSegment);
+
+        // Check that the indexes exist in in-memory RSM.
+        for (RemoteStorageManager.IndexType indexType : RemoteStorageManager.IndexType.values()) {
+            boolean containsIndex = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForIndex(segmentMetadata, indexType));
+            Assertions.assertTrue(containsIndex);
+        }
+    }
+
+    private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() {
+        TopicIdPartition topicPartition = new TopicIdPartition(Uuid.randomUuid(), TP);
+        RemoteLogSegmentId id = new RemoteLogSegmentId(topicPartition, Uuid.randomUuid());
+        return new RemoteLogSegmentMetadata(id, 100L, 200L, System.currentTimeMillis(), 0,
+                System.currentTimeMillis(), 100, Collections.singletonMap(1, 100L));
+    }
+
+    @Test
+    public void testFetchLogSegmentIndexes() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        int segSize = 100;
+        LogSegmentData logSegmentData = createLogSegmentData(segSize);
+
+        // Copy the segment
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check segment data exists for the copied segment.
+        try (InputStream segmentStream = rsm.fetchLogSegment(segmentMetadata, 0)) {
+            checkContentSame(segmentStream, logSegmentData.logSegment());
+        }
+
+        HashMap<RemoteStorageManager.IndexType, Path> expectedIndexToPaths = new HashMap<>();
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.OFFSET, logSegmentData.offsetIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TIMESTAMP, logSegmentData.timeIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TRANSACTION, logSegmentData.txnIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, logSegmentData.producerSnapshotIndex());
+
+        // Check all segment indexes exist for the copied segment.
+        for (Map.Entry<RemoteStorageManager.IndexType, Path> entry : expectedIndexToPaths.entrySet()) {
+            RemoteStorageManager.IndexType indexType = entry.getKey();
+            Path indexPath = entry.getValue();
+            log.info("Fetching index type: {}, indexPath: {}", indexType, indexPath);

Review comment:
       Is this logging needed? Does it need to be in info level?

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCacheTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * This class covers basic unit tests for {@link InmemoryRemoteLogMetadataManager}. InmemoryRemoteLogMetadataManager is
+ * used only in integration tests but not in production code. It mostly uses {@link RemoteLogMetadataCache} and it has
+ * broad test coverage with {@link RemoteLogMetadataCacheTest}.
+ */
+public class InmemoryRemoteLogMetadataManagerTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testFetchSegments() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and this segment should not be available.
+        Map<Integer, Long> segmentLeaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 101L, 200L, -1L, BROKER_ID_0,
+                time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segmentMetadata);
+
+        // Search should not return the above segment.
+        Assertions.assertFalse(rlmm.remoteLogSegmentMetadata(TP0, 0, 150).isPresent());
+
+        // 2.Move that segment to COPY_SEGMENT_FINISHED state and this segment should be available.
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+        RemoteLogSegmentMetadata expectedSegmentMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> segmentMetadataForOffset150 = rlmm.remoteLogSegmentMetadata(TP0, 0, 150);
+        Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadataForOffset150);
+    }
+
+    @Test
+    public void testRemotePartitionDeletion() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // Create remote log segment metadata and add them to RLMM.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segmentLeaderEpochs = new HashMap<>();
+        segmentLeaderEpochs.put(0, 0L);
+        segmentLeaderEpochs.put(1, 20L);
+        segmentLeaderEpochs.put(2, 50L);
+        segmentLeaderEpochs.put(3, 80L);
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segmentMetadata);
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(
+                segmentId, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+
+        RemoteLogSegmentMetadata expectedSegMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate);
+
+        // Check that the seg exists in RLMM
+        Optional<RemoteLogSegmentMetadata> segMetadataForOffset30Epoch1 = rlmm.remoteLogSegmentMetadata(TP0, 1, 30L);
+        Assertions.assertEquals(Optional.of(expectedSegMetadata), segMetadataForOffset30Epoch1);
+
+        // Mark the partition for deletion. RLMM should clear all its internal state for that partition.

Review comment:
       At this point, RLMM hasn't cleared all its internal state yet.

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100

Review comment:
       Could we make it clear this is for offset range?

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCacheTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * This class covers basic unit tests for {@link InmemoryRemoteLogMetadataManager}. InmemoryRemoteLogMetadataManager is
+ * used only in integration tests but not in production code. It mostly uses {@link RemoteLogMetadataCache} and it has
+ * broad test coverage with {@link RemoteLogMetadataCacheTest}.
+ */
+public class InmemoryRemoteLogMetadataManagerTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testFetchSegments() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and this segment should not be available.
+        Map<Integer, Long> segmentLeaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 101L, 200L, -1L, BROKER_ID_0,
+                time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segmentMetadata);
+
+        // Search should not return the above segment.
+        Assertions.assertFalse(rlmm.remoteLogSegmentMetadata(TP0, 0, 150).isPresent());
+
+        // 2.Move that segment to COPY_SEGMENT_FINISHED state and this segment should be available.
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+        RemoteLogSegmentMetadata expectedSegmentMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> segmentMetadataForOffset150 = rlmm.remoteLogSegmentMetadata(TP0, 0, 150);
+        Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadataForOffset150);
+    }
+
+    @Test
+    public void testRemotePartitionDeletion() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // Create remote log segment metadata and add them to RLMM.
+
+        // segment 0
+        // 0-100

Review comment:
       Could we make it clear this is for offset range?

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

Review comment:
       It would be useful to document the meaning of the following table.

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segment0LeaderEpochs = new HashMap<>();
+        segment0LeaderEpochs.put(0, 0L);
+        segment0LeaderEpochs.put(1, 20L);
+        segment0LeaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs);
+        cache.addCopyInProgressSegment(segment0Metadata);
+
+        // We should not get this as the segment is still getting copied and it is not yet considered successful until
+        // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate(
+                segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segment0Update);
+        RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update);
+
+        // segment 1
+        // 101 - 200
+        // no changes in leadership with in this segment
+        // leader epochs (2, 101)
+        Map<Integer, Long> segment1LeaderEpochs = Collections.singletonMap(2, 101L);
+        RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 2
+        // 201 - 300
+        // moved to epoch 3 in between
+        // leader epochs (2, 201), (3, 240)
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(2, 201L);
+        segment2LeaderEpochs.put(3, 240L);
+        RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 3
+        // 250 - 400
+        // leader epochs (3, 250), (4, 370)
+        Map<Integer, Long> segment3LeaderEpochs = new HashMap<>();
+        segment3LeaderEpochs.put(3, 250L);
+        segment3LeaderEpochs.put(4, 370L);
+        RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        // Four segments are added with different boundaries and leader epochs.
+        // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset)  for different
+        // epochs and offsets
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        HashMap<EpochOffset, RemoteLogSegmentMetadata> expectedEpochOffsetToSegmentMetadata = new HashMap<>();
+        // Existing metadata entries.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata);
+
+        // Non existing metadata entries.
+        // Search for offset 110, epoch 1, and it should not exist.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null);
+        // Search for non existing offset 401, epoch 4.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null);
+        // Search for non existing epoch 5.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null);
+
+        for (Map.Entry<EpochOffset, RemoteLogSegmentMetadata> entry : expectedEpochOffsetToSegmentMetadata.entrySet()) {
+            EpochOffset epochOffset = entry.getKey();
+            Optional<RemoteLogSegmentMetadata> segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset);
+            RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue();
+            log.info("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata,

Review comment:
       Is this logging needed? Ditto below.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610944855



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

Review comment:
       Fixed. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r608822419



##########
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) {

Review comment:
       Yes, it can be null. It is called from [here](https://github.com/apache/kafka/pull/10218/files#diff-3724bb53d7ab4bc5a6ec4e1ab4c91c47bf90e4166d881f7706e2adc1848a5d16R293)




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609372137



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segment0LeaderEpochs = new HashMap<>();
+        segment0LeaderEpochs.put(0, 0L);
+        segment0LeaderEpochs.put(1, 20L);
+        segment0LeaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs);
+        cache.addCopyInProgressSegment(segment0Metadata);
+
+        // We should not get this as the segment is still getting copied and it is not yet considered successful until
+        // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate(
+                segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segment0Update);
+        RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update);
+
+        // segment 1
+        // 101 - 200
+        // no changes in leadership with in this segment
+        // leader epochs (2, 101)
+        Map<Integer, Long> segment1LeaderEpochs = Collections.singletonMap(2, 101L);
+        RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 2
+        // 201 - 300
+        // moved to epoch 3 in between
+        // leader epochs (2, 201), (3, 240)
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(2, 201L);
+        segment2LeaderEpochs.put(3, 240L);
+        RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 3
+        // 250 - 400
+        // leader epochs (3, 250), (4, 370)
+        Map<Integer, Long> segment3LeaderEpochs = new HashMap<>();
+        segment3LeaderEpochs.put(3, 250L);
+        segment3LeaderEpochs.put(4, 370L);
+        RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        // Four segments are added with different boundaries and leader epochs.
+        // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset)  for different
+        // epochs and offsets
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        HashMap<EpochOffset, RemoteLogSegmentMetadata> expectedEpochOffsetToSegmentMetadata = new HashMap<>();
+        // Existing metadata entries.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata);
+
+        // Non existing metadata entries.
+        // Search for offset 110, epoch 1, and it should not exist.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null);
+        // Search for non existing offset 401, epoch 4.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null);
+        // Search for non existing epoch 5.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null);
+
+        for (Map.Entry<EpochOffset, RemoteLogSegmentMetadata> entry : expectedEpochOffsetToSegmentMetadata.entrySet()) {
+            EpochOffset epochOffset = entry.getKey();
+            Optional<RemoteLogSegmentMetadata> segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset);
+            RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue();
+            log.info("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata,
+                    expectedSegmentMetadata);
+            if (expectedSegmentMetadata != null) {
+                Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadata);
+            } else {
+                Assertions.assertFalse(segmentMetadata.isPresent());
+            }
+        }
+
+        // Update segment with state as DELETE_SEGMENT_STARTED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        // Update segment with state as DELETE_SEGMENT_FINISHED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        //  Search for cache.highestLogOffset(leaderEpoch) for all the leader epochs
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        Map<Integer, Long> expectedEpochToHighestOffset = new HashMap<>();
+        expectedEpochToHighestOffset.put(0, 19L);

Review comment:
       `highestLogOffset` can contain the deleted segments. `highestLogOffset` means the highest offset up to which the segments have been copied. Pl take a look at the [comment](https://github.com/apache/kafka/pull/10218#discussion_r609369253).




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-812572844


   @junrao @kowshik thanks for your review and comments. In the latest commit, I refactored the code with better abstractions and addressed the review comments.


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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r595492577



##########
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
+            // Wanted to keep this logic simple here by taking null for srcState, instead of creating one more state like
+            // COPY_SEGMENT_NOT_STARTED and have the null check by caller and pass that state.
+            return targetState == COPY_SEGMENT_STARTED;
+        } else if (srcState == targetState) {

Review comment:
       1. Will it be useful to place the implementation of this validation in a separate module, so that it can be reused with `RLMMWithTopicStorage` in the future?
   2. Suggestion from the standpoint of code readability/efficiency: Would it make sense to replace the `if-else` logic by looking up from a `Map< RemoteLogSegmentState, Set< RemoteLogSegmentState>>` where key is the source state and value is a set of allowed target states?
   
   




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610494457



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

Review comment:
       Ok, I think this is fine then.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596203582



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

Review comment:
       `add` -> adding a new entry.
   `put` -> add or update. 
   `putRemotePartitionDeleteMetadata` is used for both add or update the `RemotePartitionDeleteMetadata`.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-813897020


   Thanks @junrao for the review.  Addressed them with the latest [commit](https://github.com/apache/kafka/pull/10218/commits/83050f3e4c03c5ac58c0ba2a9d8fcbf49661a1a9). 


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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r599001851



##########
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);
+        }
+

Review comment:
       In this method, we allow for existing entries in `idToSegmentMetadata` to be replaced, even if the `state()` of the existing and new entries are the same. Is that intentional?




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610494694



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

Review comment:
       Sounds good




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609369903



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100

Review comment:
       Updated.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609372257



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class RemoteLogMetadataCacheTest {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class);
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testSegmentsLifeCycleInCache() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+        // Create remote log segment metadata and add them to RemoteLogMetadataCache.
+
+        // segment 0
+        // 0-100
+        // leader epochs (0,0), (1,20), (2,80)
+        Map<Integer, Long> segment0LeaderEpochs = new HashMap<>();
+        segment0LeaderEpochs.put(0, 0L);
+        segment0LeaderEpochs.put(1, 20L);
+        segment0LeaderEpochs.put(2, 80L);
+        RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L,
+                -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs);
+        cache.addCopyInProgressSegment(segment0Metadata);
+
+        // We should not get this as the segment is still getting copied and it is not yet considered successful until
+        // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate(
+                segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segment0Update);
+        RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update);
+
+        // segment 1
+        // 101 - 200
+        // no changes in leadership with in this segment
+        // leader epochs (2, 101)
+        Map<Integer, Long> segment1LeaderEpochs = Collections.singletonMap(2, 101L);
+        RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 2
+        // 201 - 300
+        // moved to epoch 3 in between
+        // leader epochs (2, 201), (3, 240)
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(2, 201L);
+        segment2LeaderEpochs.put(3, 240L);
+        RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // segment 3
+        // 250 - 400
+        // leader epochs (3, 250), (4, 370)
+        Map<Integer, Long> segment3LeaderEpochs = new HashMap<>();
+        segment3LeaderEpochs.put(3, 250L);
+        segment3LeaderEpochs.put(4, 370L);
+        RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        // Four segments are added with different boundaries and leader epochs.
+        // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset)  for different
+        // epochs and offsets
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        HashMap<EpochOffset, RemoteLogSegmentMetadata> expectedEpochOffsetToSegmentMetadata = new HashMap<>();
+        // Existing metadata entries.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata);
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata);
+
+        // Non existing metadata entries.
+        // Search for offset 110, epoch 1, and it should not exist.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null);
+        // Search for non existing offset 401, epoch 4.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null);
+        // Search for non existing epoch 5.
+        expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null);
+
+        for (Map.Entry<EpochOffset, RemoteLogSegmentMetadata> entry : expectedEpochOffsetToSegmentMetadata.entrySet()) {
+            EpochOffset epochOffset = entry.getKey();
+            Optional<RemoteLogSegmentMetadata> segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset);
+            RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue();
+            log.info("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata,
+                    expectedSegmentMetadata);
+            if (expectedSegmentMetadata != null) {
+                Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadata);
+            } else {
+                Assertions.assertFalse(segmentMetadata.isPresent());
+            }
+        }
+
+        // Update segment with state as DELETE_SEGMENT_STARTED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        // Update segment with state as DELETE_SEGMENT_FINISHED.
+        // It should not be available when we search for that segment.
+        cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1));
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent());
+
+        //////////////////////////////////////////////////////////////////////////////////////////
+        //  Search for cache.highestLogOffset(leaderEpoch) for all the leader epochs
+        //////////////////////////////////////////////////////////////////////////////////////////
+
+        Map<Integer, Long> expectedEpochToHighestOffset = new HashMap<>();
+        expectedEpochToHighestOffset.put(0, 19L);
+        expectedEpochToHighestOffset.put(1, 79L);
+        expectedEpochToHighestOffset.put(2, 239L);
+        expectedEpochToHighestOffset.put(3, 369L);
+        expectedEpochToHighestOffset.put(4, 400L);
+
+        for (Map.Entry<Integer, Long> entry : expectedEpochToHighestOffset.entrySet()) {
+            Integer epoch = entry.getKey();
+            Long expectedOffset = entry.getValue();
+            Optional<Long> offset = cache.highestOffsetForEpoch(epoch);
+            log.info("Fetching highest offset for epoch: {} , returned: {} , expected: {}", epoch, offset, expectedOffset);
+            Assertions.assertEquals(Optional.of(expectedOffset), offset);
+        }
+
+        // Search for non existing leader epoch
+        Optional<Long> highestOffsetForEpoch5 = cache.highestOffsetForEpoch(5);
+        Assertions.assertFalse(highestOffsetForEpoch5.isPresent());
+    }
+
+    private RemoteLogSegmentMetadata createSegmentUpdateWithState(RemoteLogMetadataCache cache,
+                                                                  Map<Integer, Long> segmentLeaderEpochs,
+                                                                  long startOffset,
+                                                                  long endOffset,
+                                                                  RemoteLogSegmentState state)
+            throws RemoteResourceNotFoundException {
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, startOffset, endOffset, -1L,
+                BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        cache.addCopyInProgressSegment(segmentMetadata);
+
+        RemoteLogSegmentMetadataUpdate segMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId,
+                time.milliseconds(), state, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segMetadataUpdate);
+
+        return segmentMetadata.createWithUpdates(segMetadataUpdate);
+    }
+
+    @Test
+    public void testCacheSegmentWithCopySegmentStartedState() {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 50L, -1L, BROKER_ID_0,
+                time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        cache.addCopyInProgressSegment(segmentMetadata);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> segMetadataForOffset0Epoch0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(segMetadataForOffset0Epoch0.isPresent());
+
+        // cache.listRemoteLogSegments APIs should contain the above segment.
+        checkListSegments(cache, 0, segmentMetadata);
+    }
+
+    @Test
+    public void testCacheSegmentWithCopySegmentFinishedState() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 101L),
+                101L, 200L, RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> segMetadataForOffset150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(Optional.of(segmentMetadata), segMetadataForOffset150);
+
+        // cache.listRemoteLogSegments should contain the above segments.
+        checkListSegments(cache, 0, segmentMetadata);
+    }
+
+    @Test
+    public void testCacheSegmentWithDeleteSegmentStartedState() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 201L),
+                201L, 300L, RemoteLogSegmentState.DELETE_SEGMENT_STARTED);
+
+        // Search should not return the above segment as their leader epoch state is cleared.
+        Optional<RemoteLogSegmentMetadata> segmentMetadataForOffset250Epoch0 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertFalse(segmentMetadataForOffset250Epoch0.isPresent());
+
+        checkListSegments(cache, 0, segmentMetadata);
+    }
+
+    @Test
+    public void testCacheSegmentsWithDeleteSegmentFinishedState() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 301L),
+                301L, 400L, RemoteLogSegmentState.DELETE_SEGMENT_STARTED);
+
+        // Search should not return the above segment as their leader epoch state is cleared.
+        Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 350).isPresent());
+
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(),
+                time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1);
+        cache.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+
+        // listRemoteLogSegments(0) and listRemoteLogSegments() should not contain the above segment.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        Assertions.assertFalse(cache.listAllRemoteLogSegments().hasNext());
+    }
+
+    @Test
+    public void testCacheListSegments() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Create a few segments and add them to the cache.
+        RemoteLogSegmentMetadata segment0 = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 0L), 0, 100,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+        RemoteLogSegmentMetadata segment1 = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 101L), 101, 200,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+        Map<Integer, Long> segment2LeaderEpochs = new HashMap<>();
+        segment2LeaderEpochs.put(0, 201L);
+        segment2LeaderEpochs.put(1, 301L);
+        RemoteLogSegmentMetadata segment2 = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201, 400,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+
+        // listRemoteLogSegments(0) and listAllRemoteLogSegments() should contain all the above segments.
+        List<RemoteLogSegmentMetadata> expectedSegmentsForEpoch0 = Arrays.asList(segment0, segment1, segment2);
+        Assertions.assertTrue(TestUtils.sameElementsWithOrder(cache.listRemoteLogSegments(0),
+                expectedSegmentsForEpoch0.iterator()));
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(cache.listAllRemoteLogSegments(),
+                expectedSegmentsForEpoch0.iterator()));
+
+        // listRemoteLogSegments(0) should contain only segment2.

Review comment:
       Done




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609351020



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+public class InmemoryRemoteStorageManagerTest {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManagerTest.class);
+
+    private static final TopicPartition TP = new TopicPartition("foo", 1);
+    private static final File DIR = TestUtils.tempDirectory("inmem-rsm-");
+    private static final Random RANDOM = new Random();
+
+    @Test
+    public void testCopyLogSegment() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        LogSegmentData logSegmentData = createLogSegmentData();
+        // Copy all the segment data.
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check that the segment data exists in in-memory RSM.
+        boolean containsSegment = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForSegment(segmentMetadata));
+        Assertions.assertTrue(containsSegment);
+
+        // Check that the indexes exist in in-memory RSM.
+        for (RemoteStorageManager.IndexType indexType : RemoteStorageManager.IndexType.values()) {
+            boolean containsIndex = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForIndex(segmentMetadata, indexType));
+            Assertions.assertTrue(containsIndex);
+        }
+    }
+
+    private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() {
+        TopicIdPartition topicPartition = new TopicIdPartition(Uuid.randomUuid(), TP);
+        RemoteLogSegmentId id = new RemoteLogSegmentId(topicPartition, Uuid.randomUuid());
+        return new RemoteLogSegmentMetadata(id, 100L, 200L, System.currentTimeMillis(), 0,
+                System.currentTimeMillis(), 100, Collections.singletonMap(1, 100L));
+    }
+
+    @Test
+    public void testFetchLogSegmentIndexes() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        int segSize = 100;
+        LogSegmentData logSegmentData = createLogSegmentData(segSize);
+
+        // Copy the segment
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check segment data exists for the copied segment.
+        try (InputStream segmentStream = rsm.fetchLogSegment(segmentMetadata, 0)) {
+            checkContentSame(segmentStream, logSegmentData.logSegment());
+        }
+
+        HashMap<RemoteStorageManager.IndexType, Path> expectedIndexToPaths = new HashMap<>();
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.OFFSET, logSegmentData.offsetIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TIMESTAMP, logSegmentData.timeIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TRANSACTION, logSegmentData.txnIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, logSegmentData.producerSnapshotIndex());
+
+        // Check all segment indexes exist for the copied segment.
+        for (Map.Entry<RemoteStorageManager.IndexType, Path> entry : expectedIndexToPaths.entrySet()) {
+            RemoteStorageManager.IndexType indexType = entry.getKey();
+            Path indexPath = entry.getValue();
+            log.info("Fetching index type: {}, indexPath: {}", indexType, indexPath);

Review comment:
       We may have this as debug level by default. It will be helpful to see for which entry the test is failed. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610454559



##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,46 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     */
+    public static <T> boolean sameElementsWithOrder(Iterator<T> iterator1,

Review comment:
       Existing code looks easy to read/comprehend and no multiple calls to hasNext().
   How about the below code after removing inline variables in the existing code?
   
   ```
   while (iterator1.hasNext()) {
       if (!iterator2.hasNext()) {
           return false;
       }
   
       if (!Objects.equals(iterator1.next(), iterator2.next())) {
           return false;
       }
   }
   
   return !iterator2.hasNext();
   ```




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607562925



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}.
+     *
+     * @param leaderEpoch leader epoch for the given offset
+     * @param offset      offset
+     * @return the requested remote log segment metadata if it exists.
+     */
+    public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) {
+        RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch);
+
+        if (remoteLogLeaderEpochState == null) {
+            return Optional.empty();
+        }
+
+        // Look for floor entry as the given offset may exist in this entry.
+        RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset);
+        if (remoteLogSegmentId == null) {
+            // If the offset is lower than the minimum offset available in metadata then return empty.
+            return Optional.empty();
+        }
+
+        RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId);
+        // Check whether the given offset with leaderEpoch exists in this segment.
+        // Check for epoch's offset boundaries with in this segment.
+        //      1. Get the next epoch's start offset -1 if exists
+        //      2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset.
+        Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch);
+        long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset();
+
+        // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state.
+        // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no

Review comment:
       Updated the comment.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-817018916


   Thanks @junrao for the comment, addressed with the commit [317be7a](https://github.com/apache/kafka/pull/10218/commits/317be7abe50264b6ed09bb85f36eacdc6b09599f). 


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



[GitHub] [kafka] junrao merged pull request #10218: KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #10218:
URL: https://github.com/apache/kafka/pull/10218


   


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r609369434



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

Review comment:
       Added a note.

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache;
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCacheTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * This class covers basic unit tests for {@link InmemoryRemoteLogMetadataManager}. InmemoryRemoteLogMetadataManager is
+ * used only in integration tests but not in production code. It mostly uses {@link RemoteLogMetadataCache} and it has
+ * broad test coverage with {@link RemoteLogMetadataCacheTest}.
+ */
+public class InmemoryRemoteLogMetadataManagerTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID_0 = 0;
+    private static final int BROKER_ID_1 = 1;
+
+    private final Time time = new MockTime(1);
+
+    @Test
+    public void testFetchSegments() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and this segment should not be available.
+        Map<Integer, Long> segmentLeaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 101L, 200L, -1L, BROKER_ID_0,
+                time.milliseconds(), SEG_SIZE, segmentLeaderEpochs);
+        rlmm.addRemoteLogSegmentMetadata(segmentMetadata);
+
+        // Search should not return the above segment.
+        Assertions.assertFalse(rlmm.remoteLogSegmentMetadata(TP0, 0, 150).isPresent());
+
+        // 2.Move that segment to COPY_SEGMENT_FINISHED state and this segment should be available.
+        RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1);
+        rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate);
+        RemoteLogSegmentMetadata expectedSegmentMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> segmentMetadataForOffset150 = rlmm.remoteLogSegmentMetadata(TP0, 0, 150);
+        Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadataForOffset150);
+    }
+
+    @Test
+    public void testRemotePartitionDeletion() throws Exception {
+        InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager();
+
+        // Create remote log segment metadata and add them to RLMM.
+
+        // segment 0
+        // 0-100

Review comment:
       Done




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610305248



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

Review comment:
       Is this method expected to be idempotent?
   Note: this comment is related to my other comment: https://github.com/apache/kafka/pull/10218#discussion_r610298531.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606250883



##########
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);
+        }
+

Review comment:
       Yes, it can happen to generate an event with the same state incase of retries.  




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607597875



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       One way to do that is to clear the entry when the respective `RemoteLogLeaderEpochState` is empty. That means all the segments reached `DELETE_SEGMENT_FINISHED` state. 
   This is not currently addressed. I plan to look into it when we integrate these APIs with RemoteLogManager by exploring other options too. 
   




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592070871



##########
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:
       This can happen in race condition when this method is queried while it was getting added. It may not happen in practice but it is good to have these checks.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610940157



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+public class InmemoryRemoteStorageManagerTest {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManagerTest.class);
+
+    private static final TopicPartition TP = new TopicPartition("foo", 1);
+    private static final File DIR = TestUtils.tempDirectory("inmem-rsm-");
+    private static final Random RANDOM = new Random();
+
+    @Test
+    public void testCopyLogSegment() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        LogSegmentData logSegmentData = createLogSegmentData();
+        // Copy all the segment data.
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check that the segment data exists in in-memory RSM.
+        boolean containsSegment = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForSegment(segmentMetadata));
+        Assertions.assertTrue(containsSegment);
+
+        // Check that the indexes exist in in-memory RSM.
+        for (RemoteStorageManager.IndexType indexType : RemoteStorageManager.IndexType.values()) {
+            boolean containsIndex = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForIndex(segmentMetadata, indexType));
+            Assertions.assertTrue(containsIndex);
+        }
+    }
+
+    private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() {
+        TopicIdPartition topicPartition = new TopicIdPartition(Uuid.randomUuid(), TP);
+        RemoteLogSegmentId id = new RemoteLogSegmentId(topicPartition, Uuid.randomUuid());
+        return new RemoteLogSegmentMetadata(id, 100L, 200L, System.currentTimeMillis(), 0,
+                System.currentTimeMillis(), 100, Collections.singletonMap(1, 100L));
+    }
+
+    @Test
+    public void testFetchLogSegmentIndexes() throws Exception {
+        InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager();
+        RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata();
+        int segSize = 100;
+        LogSegmentData logSegmentData = createLogSegmentData(segSize);
+
+        // Copy the segment
+        rsm.copyLogSegmentData(segmentMetadata, logSegmentData);
+
+        // Check segment data exists for the copied segment.
+        try (InputStream segmentStream = rsm.fetchLogSegment(segmentMetadata, 0)) {
+            checkContentSame(segmentStream, logSegmentData.logSegment());
+        }
+
+        HashMap<RemoteStorageManager.IndexType, Path> expectedIndexToPaths = new HashMap<>();
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.OFFSET, logSegmentData.offsetIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TIMESTAMP, logSegmentData.timeIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.TRANSACTION, logSegmentData.txnIndex());
+        expectedIndexToPaths.put(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, logSegmentData.producerSnapshotIndex());
+
+        // Check all segment indexes exist for the copied segment.
+        for (Map.Entry<RemoteStorageManager.IndexType, Path> entry : expectedIndexToPaths.entrySet()) {
+            RemoteStorageManager.IndexType indexType = entry.getKey();
+            Path indexPath = entry.getValue();
+            log.info("Fetching index type: {}, indexPath: {}", indexType, indexPath);

Review comment:
       Yes, it is done. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591603442



##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", rlsmUpdate);
+            // remove this entry when the state is moved to delete_segment_finished
+            Map<Integer, Long> leaderEpochs = rlsm.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 Long highestLogOffset(int leaderEpoch) {

Review comment:
       No, it is not highestSegmentStartOffset but it is the highest log offset for the given leader epoch.
   Nice catch! We need to give the max endoffset of all the segments for the given leader epoch. 




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592123658



##########
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:
       Is it useful to add a check against it?




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



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

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r590742564



##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", rlsmUpdate);
+            // remove this entry when the state is moved to delete_segment_finished
+            Map<Integer, Long> leaderEpochs = rlsm.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:
       Could we just get the segment list from `idToSegmentMetadata.values()`?

##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {
+            log.debug("Cleaning up the state for : [{}]", rlsmUpdate);
+            // remove this entry when the state is moved to delete_segment_finished
+            Map<Integer, Long> leaderEpochs = rlsm.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 Long highestLogOffset(int leaderEpoch) {

Review comment:
       highestLogOffset => highestSegmentStartOffset ?
   

##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);

Review comment:
       Hmm, it's possible for a segment to transition to DELETE_SEGMENT_STARTED here. Should those segments still be added?

##########
File path: clients/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 imemory cache of remote log segment metadata. This maintains the lineage of segments

Review comment:
       typo imemory

##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == RemoteLogSegmentState.DELETE_SEGMENT_FINISHED) {

Review comment:
       Once a segment is in DELETE_SEGMENT_STARTED state, the corresponding segment could be gone any time after that. So, it seems that we should remove the segment from leaderEpochToOffsetToId once it's in DELETE_SEGMENT_STARTED?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager {

Review comment:
       Could we add a comment to this class?

##########
File path: clients/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 imemory 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 null.

Review comment:
       We are not returning null here.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606257469



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCacheTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class RemoteLogMetadataCacheTest {
+
+    private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(),
+            new TopicPartition("foo", 0));
+    private static final int SEG_SIZE = 1024 * 1024;
+    private static final int BROKER_ID = 0;
+
+    @Test
+    public void testCacheSegmentsWithDifferentStates() throws Exception {
+        RemoteLogMetadataCache cache = new RemoteLogMetadataCache();
+
+        // Add segments with different states and check cache.remoteLogSegmentMetadata(int leaderEpoch, long offset)
+        // cache.listRemoteLogSegments(int leaderEpoch), and cache.listAllRemoteLogSegments().
+
+        // =============================================================================================================
+        // 1.Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the
+        // segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg0leaderEpochs = Collections.singletonMap(0, 0L);
+        RemoteLogSegmentId seg0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata segCopyInProgress = new RemoteLogSegmentMetadata(seg0Id, 0L, 50L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg0leaderEpochs);
+        cache.addToInProgress(segCopyInProgress);
+
+        // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED.
+        Optional<RemoteLogSegmentMetadata> seg0s0e0 = cache.remoteLogSegmentMetadata(0, 0);
+        Assertions.assertFalse(seg0s0e0.isPresent());
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment, it will be empty.
+        Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext());
+        // But cache.listRemoteLogSegments() should contain the above segment.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Collections.singletonList(segCopyInProgress));
+
+        // =============================================================================================================
+        // 2.Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg1leaderEpochs = Collections.singletonMap(0, 101L);
+        RemoteLogSegmentId seg1Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg1 = new RemoteLogSegmentMetadata(seg1Id, 101L, 200L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg1leaderEpochs);
+        cache.addToInProgress(seg1);
+        RemoteLogSegmentMetadataUpdate seg1Update = new RemoteLogSegmentMetadataUpdate(seg1Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg1Update);
+        RemoteLogSegmentMetadata segCopyFinished = seg1.createRemoteLogSegmentWithUpdates(seg1Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg1S150 = cache.remoteLogSegmentMetadata(0, 150);
+        Assertions.assertEquals(seg1.createRemoteLogSegmentWithUpdates(seg1Update), seg1S150.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should not contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Collections.singletonList(segCopyFinished));
+        // But cache.listRemoteLogSegments() should contain both the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(), Arrays.asList(segCopyInProgress, segCopyFinished));
+
+        // =============================================================================================================
+        // 3.Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg2leaderEpochs = Collections.singletonMap(0, 201L);
+        RemoteLogSegmentId seg2Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg2 = new RemoteLogSegmentMetadata(seg2Id, 201L, 300L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg2leaderEpochs);
+        cache.addToInProgress(seg2);
+        RemoteLogSegmentMetadataUpdate seg2Update = new RemoteLogSegmentMetadataUpdate(seg2Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);
+        cache.updateRemoteLogSegmentMetadata(seg2Update);
+        RemoteLogSegmentMetadata segDeleteStarted = seg2.createRemoteLogSegmentWithUpdates(seg2Update);
+
+        // Search should return the above segment.
+        Optional<RemoteLogSegmentMetadata> seg2S250 = cache.remoteLogSegmentMetadata(0, 250);
+        Assertions.assertEquals(seg2.createRemoteLogSegmentWithUpdates(seg2Update), seg2S250.orElse(null));
+
+        // cache.listRemoteLogSegments(0) should contain the above segment.
+        checkContainsAll(cache.listRemoteLogSegments(0), Arrays.asList(segCopyFinished, segDeleteStarted));
+        // But cache.listRemoteLogSegments() should contain all the segments.
+        checkContainsAll(cache.listAllRemoteLogSegments(),
+                Arrays.asList(segCopyInProgress, segCopyFinished, segDeleteStarted));
+
+        // =============================================================================================================
+        // 4.Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and
+        // listing the segments.
+        // ==============================================================================================================
+        Map<Integer, Long> seg3leaderEpochs = Collections.singletonMap(0, 301L);
+        RemoteLogSegmentId seg3Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid());
+        RemoteLogSegmentMetadata seg3 = new RemoteLogSegmentMetadata(seg3Id, 301L, 400L, -1L, BROKER_ID,
+                System.currentTimeMillis(), SEG_SIZE, seg3leaderEpochs);
+        cache.addToInProgress(seg3);
+        RemoteLogSegmentMetadataUpdate seg3Update1 = new RemoteLogSegmentMetadataUpdate(seg3Id,
+                System.currentTimeMillis(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID);

Review comment:
       Sure, I will add that.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-806359943


   @kowshik I discussed the proposed changes in the call on 23rd,  and I mentioned that PR is not updated with those changes. I will let you know once those changes are pushed into this PR.  


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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596055570



##########
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:
       Good point! It will clear the values which are empty maps. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r596204864



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

Review comment:
       It was deliberate not to add locking semantics for now. We will add them once we have the respective changes using these classes. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592075662



##########
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 may not occur 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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592124191



##########
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:
       Sure




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592096637



##########
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:
       It allows any state other than `COPY_SEGMENT_STARTED`, that is why we are checking only for this state.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607561788



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

Review comment:
       Not really needed for now. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606253804



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),

Review comment:
       We can add that. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591585302



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

Review comment:
       The plan was to use the related classes in the default RLMM implementation and move any class which is relevant only for tests to test dir later. I am +1 to have this as a separate module. I will update with those changes. 
   




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r591591507



##########
File path: clients/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 imemory 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 null.
+            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);
+        if (rlsm == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment metadata found for : "
+                                                      + remoteLogSegmentId);
+        }
+
+        log.debug("Updating remote log segment metadata: [{}]", rlsmUpdate);
+        RemoteLogSegmentMetadata updatedRlsm = rlsm.createRemoteLogSegmentWithUpdates(rlsmUpdate);
+        idToSegmentMetadata.put(remoteLogSegmentId, updatedRlsm);
+        if (rlsmUpdate.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) {
+            remoteLogSegmentIdInProgress.remove(remoteLogSegmentId);
+            addRemoteLogSegmentMetadata(updatedRlsm);
+        }
+
+        if (rlsmUpdate.state() == 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. 




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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610293058



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

Review comment:
       Hmm here we assume that `id` should be present in the provided `idToSegmentMetadata`. Due to programming error, or other reasons, the caller may not be able to ensure this. Would it be safer if we instead threw whenever `id` is absent in `idToSegmentMetadata`  to catch that case?

##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,46 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     */
+    public static <T> boolean sameElementsWithOrder(Iterator<T> iterator1,

Review comment:
       Here is a slightly simpler version:
   ```
    while (iterator1.hasNext() && iterator2.hasNext()) {
        if (!Objects.equals(iterator1.next(), iterator2.next())) {
               return false;
       }
   }
   
   return !iterator1.hasNext() && !iterator2.hasNext();
   ```

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

Review comment:
       Is this method expected to be idempotent?

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

Review comment:
       The add call won't replace an existing element with the same `remoteLogSegmentId`. Is that expected?
   For example, what happens if `addCopyInProgressSegment` is called twice but this line doesn't replace the existing entry?

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

Review comment:
       nit: add one whitespace at the end after "...state" 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606249948



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),
+                    Files.readAllBytes(logSegmentData.logSegment().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Offset),
+                    Files.readAllBytes(logSegmentData.offsetIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Timestamp),
+                    Files.readAllBytes(logSegmentData.timeIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Transaction),
+                    Files.readAllBytes(logSegmentData.txnIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LeaderEpoch),
+                    logSegmentData.leaderEpochIndex().array());
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.ProducerSnapshot),
+                    Files.readAllBytes(logSegmentData.producerSnapshotIndex().toPath()));
+        } catch (IOException e) {
+            throw new RemoteStorageException(e.getMessage(), e);
+        }
+        log.debug("copied log segment and indexes for : {} successfully.", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition)
+            throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] for [{}]", startPosition, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        return fetchLogSegment(remoteLogSegmentMetadata, startPosition, Integer.MAX_VALUE);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition,
+                                       int endPosition) throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] and end position: [{}] for segment [{}]",
+                startPosition, endPosition, remoteLogSegmentMetadata);
+
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        if (startPosition < 0 || endPosition < 0) {
+            throw new IllegalArgumentException("Given start position or end position must not be negative.");
+        }
+
+        if (endPosition < startPosition) {
+            throw new IllegalArgumentException("end position must be greater than start position");
+        }
+
+        String key = generateKeyForSegment(remoteLogSegmentMetadata);
+        byte[] segment = keyToLogData.get(key);
+
+        if (segment == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment found with start offset:"
+                                                      + remoteLogSegmentMetadata.startOffset() + " and id: "
+                                                      + remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+
+        if (startPosition >= segment.length) {
+            throw new IllegalArgumentException("start position: " + startPosition
+                                               + " must be less than the length of the segment: " + segment.length);
+        }
+
+        // check for boundaries like given end position is more than the length, length should never be more than the
+        // existing segment size.
+        int length = Math.min(segment.length - 1, endPosition) - startPosition + 1;

Review comment:
       It takes `Math.min(endPosition, segment.length)`. So, no need to have that check. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607556778



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

Review comment:
       Looks like autoformatter changed it. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r606249998



##########
File path: remote-storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class is an implementation of {@link RemoteStorageManager} backed by inmemory store.
+ */
+public class InmemoryRemoteStorageManager implements RemoteStorageManager {
+    private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class);
+
+    // map of key to log data, which can be segment or any of its indexes.
+    private Map<String, byte[]> keyToLogData = new ConcurrentHashMap<>();
+
+    public InmemoryRemoteStorageManager() {
+    }
+
+    static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment";
+    }
+
+    static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                      IndexType indexType) {
+        return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString();
+    }
+
+    // visible for testing.
+    boolean containsKey(String key) {
+        return keyToLogData.containsKey(key);
+    }
+
+    @Override
+    public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                   LogSegmentData logSegmentData)
+            throws RemoteStorageException {
+        log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(logSegmentData, "logSegmentData can not be null");
+        try {
+            keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata),
+                    Files.readAllBytes(logSegmentData.logSegment().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Offset),
+                    Files.readAllBytes(logSegmentData.offsetIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Timestamp),
+                    Files.readAllBytes(logSegmentData.timeIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.Transaction),
+                    Files.readAllBytes(logSegmentData.txnIndex().toPath()));
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LeaderEpoch),
+                    logSegmentData.leaderEpochIndex().array());
+            keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.ProducerSnapshot),
+                    Files.readAllBytes(logSegmentData.producerSnapshotIndex().toPath()));
+        } catch (IOException e) {
+            throw new RemoteStorageException(e.getMessage(), e);
+        }
+        log.debug("copied log segment and indexes for : {} successfully.", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition)
+            throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] for [{}]", startPosition, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        return fetchLogSegment(remoteLogSegmentMetadata, startPosition, Integer.MAX_VALUE);
+    }
+
+    @Override
+    public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                       int startPosition,
+                                       int endPosition) throws RemoteStorageException {
+        log.debug("Received fetch segment request at start position: [{}] and end position: [{}] for segment [{}]",
+                startPosition, endPosition, remoteLogSegmentMetadata);
+
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+
+        if (startPosition < 0 || endPosition < 0) {
+            throw new IllegalArgumentException("Given start position or end position must not be negative.");
+        }
+
+        if (endPosition < startPosition) {
+            throw new IllegalArgumentException("end position must be greater than start position");
+        }
+
+        String key = generateKeyForSegment(remoteLogSegmentMetadata);
+        byte[] segment = keyToLogData.get(key);
+
+        if (segment == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment found with start offset:"
+                                                      + remoteLogSegmentMetadata.startOffset() + " and id: "
+                                                      + remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+
+        if (startPosition >= segment.length) {
+            throw new IllegalArgumentException("start position: " + startPosition
+                                               + " must be less than the length of the segment: " + segment.length);
+        }
+
+        // check for boundaries like given end position is more than the length, length should never be more than the
+        // existing segment size.
+        int length = Math.min(segment.length - 1, endPosition) - startPosition + 1;
+        log.debug("Length of the segment to be sent: [{}], for segment: [{}]", length, remoteLogSegmentMetadata);
+
+        return new ByteArrayInputStream(segment, startPosition, length);
+    }
+
+    @Override
+    public InputStream fetchIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                  IndexType indexType) throws RemoteStorageException {
+        log.debug("Received fetch request for index type: [{}], segment [{}]", indexType, remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        Objects.requireNonNull(indexType, "indexType can not be null");
+
+        String key = generateKeyForIndex(remoteLogSegmentMetadata, indexType);
+        byte[] index = keyToLogData.get(key);
+        if (index == null) {
+            throw new RemoteResourceNotFoundException("No remote log segment index found with start offset:"
+                                                      + remoteLogSegmentMetadata.startOffset() + " and id: "
+                                                      + remoteLogSegmentMetadata.remoteLogSegmentId());
+        }
+
+        return new ByteArrayInputStream(index);
+    }
+
+    @Override
+    public void deleteLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException {
+        log.info("Deleting log segment for: [{}]", remoteLogSegmentMetadata);
+        Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null");
+        String segmentKey = generateKeyForSegment(remoteLogSegmentMetadata);
+        keyToLogData.remove(segmentKey);
+        for (IndexType indexType : IndexType.values()) {
+            String key = generateKeyForIndex(remoteLogSegmentMetadata, indexType);
+            keyToLogData.remove(key);
+        }
+        log.info("Deleted log segment successfully for: [{}]", remoteLogSegmentMetadata);
+    }
+
+    @Override
+    public void close() throws IOException {
+        keyToLogData = Collections.emptyMap();
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {

Review comment:
       Right.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607546803



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <table border="4">
+ * <thead border="4">
+ * <tr>
+ * <th></th>
+ * <th>COPY_SEGMENT_STARTED</th>
+ * <th>COPY_SEGMENT_FINISHED</th>
+ * <th>DELETE_SEGMENT_STARTED</th>
+ * <th>DELETE_SEGMENT_FINISHED</th>
+ * </tr>
+ * </thead>
+ * <tbody>
+ * <tr>
+ * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td>
+ * <td>No</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listRemoteLogSegments <br>(int leaderEpoch)</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * <tr>
+ * <td>listAllRemoteLogSegments()</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>Yes</td>
+ * <td>No</td>
+ * </tr>
+ * </tbody>
+ * </table>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {

Review comment:
       No. javadoc is generated for clients module with the package `/org/apache/kafka/server/log/remote/storage/ `. But this class is in `remote-storage` module.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicate it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>

Review comment:
       Sure, I will make it as simple ascii text. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610517070



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

Review comment:
       Sure, I will add a check for that. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r610940434



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

Review comment:
       Yes, I updated the PR.




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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592022166



##########
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 a prior 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?




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607580027



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogLeaderEpochState.java
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * This class represents the in-memory state of segments associated with a leader epoch. This includes the mapping of offset to
+ * segment ids and unreferenced segments which are not mapped to any offset but they exist in remote storage.
+ * <p>
+ * This is used by {@link RemoteLogMetadataCache} to track the segments for each leader epoch.
+ */
+class RemoteLogLeaderEpochState {
+
+    // It contains offset to segment ids mapping with the segment state as COPY_SEGMENT_FINISHED.
+    private final NavigableMap<Long, RemoteLogSegmentId> offsetToId = new ConcurrentSkipListMap<>();
+
+    /**
+     * It represents unreferenced segments for this leader epoch. It contains the segments still in COPY_SEGMENT_STARTED
+     * and DELETE_SEGMENT_STARTED state or these have been replaced by callers with other segments having the same
+     * start offset for the leader epoch. These will be returned by {@link RemoteLogMetadataCache#listAllRemoteLogSegments()}
+     * and {@link RemoteLogMetadataCache#listRemoteLogSegments(int leaderEpoch)} so that callers can clean them up if
+     * they still exist. These will be cleaned from the cache once they reach DELETE_SEGMENT_FINISHED state.
+     */
+    private final Set<RemoteLogSegmentId> unreferencedSegmentIds = ConcurrentHashMap.newKeySet();
+
+    // It represents the highest log offset of the segments that were updated with updateHighestLogOffset.
+    private volatile Long highestLogOffset;
+
+    /**
+     * Returns all the segments associated with this leader epoch sorted by start offset in ascending order.
+     *
+     * @param idToSegmentMetadata mapping of id to segment metadata. This will be used to get RemoteLogSegmentMetadata
+     *                            for an id to be used for sorting.
+     * @return
+     */
+    Iterator<RemoteLogSegmentMetadata> listAllRemoteLogSegments(Map<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata) {
+        // Return all the segments including unreferenced metadata.
+        int size = offsetToId.size() + unreferencedSegmentIds.size();
+        if (size == 0) {
+            return Collections.emptyIterator();
+        }
+
+        ArrayList<RemoteLogSegmentMetadata> metadataList = new ArrayList<>(size);
+        for (RemoteLogSegmentId id : offsetToId.values()) {
+            metadataList.add(idToSegmentMetadata.get(id));
+        }
+
+        if (!unreferencedSegmentIds.isEmpty()) {
+            for (RemoteLogSegmentId id : unreferencedSegmentIds) {
+                metadataList.add(idToSegmentMetadata.get(id));
+            }
+
+            // sort only when unreferenced entries exist as they are already sorted in offsetToId.
+            metadataList.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset));
+        }
+
+        return metadataList.iterator();
+    }
+
+    void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
+                                                   Long leaderEpochEndOffset) {
+        // Add the segment epochs mapping as the segment is copied successfully.
+        RemoteLogSegmentId oldEntry = offsetToId.put(startOffset, remoteLogSegmentId);
+
+        // Remove the metadata from unreferenced entries as it is successfully copied and added to the offset mapping.
+        unreferencedSegmentIds.remove(remoteLogSegmentId);
+
+        // Add the old entry to unreferenced entries as the mapping is removed for the old entry.
+        if (oldEntry != null) {
+            unreferencedSegmentIds.add(oldEntry);

Review comment:
       Yes, in the case of unclean leader election, the leader will remove the old segments for the respective leader epochs. The removal process involves removing the actual segment and updating the respective metadata of the segments. 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592248245



##########
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:
       I meant there will be an external trigger based on delete partition marker, that is responsible for deleting the segments in a partition and updating the metadata. I will remove it as it looks to create confusion.  




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r608820288



##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.log.remote.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments
+ * with respect to leader epochs.
+ * <p>
+ * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}.
+ * <p>
+ * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any
+ * segment reaching the terminal state will get cleared from this instance.
+ * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)},
+ * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those
+ * methods have different semantics to fetch the segment based on its state.
+ * <p>
+ * <ul>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}:
+ * <br>
+ * Segment in this state indicates it is not yet copied successfully. So, these segments will not be
+ * accessible for reads but these are considered for cleanups when a partition is deleted.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}:
+ * <br>
+ * Segment in this state indicates it is successfully copied and it is available for reads. So, these segments
+ * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the
+ * caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}:
+ * Segment in this state indicates it is getting deleted. That means, it is not available for reads. But it should be
+ * available for any cleanup activity like deleting segments by the caller of this class.
+ * </li>
+ * <li>
+ * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}:
+ * Segment in this state indicate it is already deleted. That means, it is not available for any activity including
+ * reads or cleanup activity. This cache will clear entries containing this state.
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * <pre>
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * |                                 | COPY_SEGMENT_STARTED | COPY_SEGMENT_FINISHED  | DELETE_SEGMENT_STARTED  | DELETE_SEGMENT_STARTED  |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | remoteLogSegmentMetadata        |        No            |           Yes          |          No             |           No            |
+ * | (int leaderEpoch, long offset)  |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listRemoteLogSegments           |        Yes           |           Yes          |          Yes            |           No            |
+ * | (int leaderEpoch)               |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listAllRemoteLogSegments()      |        Yes           |           Yes          |          Yes            |           No            |
+ * |                                 |                      |                        |                         |                         |
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * </pre>
+ * </p>
+ * <p></p>
+ */
+public class RemoteLogMetadataCache {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class);
+
+    // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED.
+    private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata
+            = new ConcurrentHashMap<>();
+
+    // It contains leader epoch to the respective entry containing the state.
+    private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>();

Review comment:
       The key is not really `id` but `epoch num`. What about `remoteLogLeaderEpochStateEntries` or `leaderEpochToState` or any other better name? 




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#issuecomment-815513359


   @junrao Thanks for your comments. Replied on comments and addressed them with the [b5c0081](https://github.com/apache/kafka/pull/10218/commits/b5c0081b6f2740606e8458a3e784994ac5a5ee85) commit. 


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



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

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r589867244



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

Review comment:
       It's a bit weird to include this in the client module. Since this is implemented in java, we could potentially create a new java module for it (like the Raft module). This reduces the size of the client jar and also avoids the inter-dependencies between java and scala.
   
   Also, is this for testing? If so, it needs to be in the test directory.

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

Review comment:
       Is this for testing? If so, it needs to be in the test directory.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592070871



##########
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:
       This can happen in race condition when this method is queried while it was getting added in `addRemoteLogSegmentMetadata`. It may not happen in practice but it is good to have these checks.




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



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

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r592577341



##########
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:
       Sure, will add comments. 




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