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


Reply via email to