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