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


Reply via email to