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


Reply via email to