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


Reply via email to