kowshik commented on a change in pull request #10218: URL: https://github.com/apache/kafka/pull/10218#discussion_r595492577
########## File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java ########## @@ -87,4 +89,27 @@ public byte id() { public static RemoteLogSegmentState forId(byte id) { return STATE_TYPES.get(id); } + + public static boolean isValidTransition(RemoteLogSegmentState srcState, RemoteLogSegmentState targetState) { + Objects.requireNonNull(targetState, "targetState can not be null"); + + if (srcState == null) { + // If the source state is null, check the target state as the initial state viz DELETE_PARTITION_MARKED + // Wanted to keep this logic simple here by taking null for srcState, instead of creating one more state like + // COPY_SEGMENT_NOT_STARTED and have the null check by caller and pass that state. + return targetState == COPY_SEGMENT_STARTED; + } else if (srcState == targetState) { Review comment: 1. Will it be useful to place the implementation of this validation in a separate module, so that it can be reused with `RLMMWithTopicStorage` in the future? 2. Suggestion from the standpoint of code readability: Would it make sense to replace the `if-else` logic by looking up from a `Map< RemoteLogSegmentState, Set< RemoteLogSegmentState>>` where key is the source state and value is a set of allowed target states? ########## File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java ########## @@ -0,0 +1,185 @@ +/* + * 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 { Review comment: We may want to think more about the locking semantics for this class and `RemoteLogMetadataCache`. Are we sure there would _not_ be use cases where we need to serialize mutations across the individually thread-safe attributes? If the answer is no, then using a fine-grained `Object` lock makes more sense because we can use it to guard critical sections. Should we evaluate this upfront? cc @junrao ########## File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java ########## @@ -83,4 +85,25 @@ public static RemotePartitionDeleteState forId(byte id) { return STATE_TYPES.get(id); } + public static boolean isValidTransition(RemotePartitionDeleteState srcState, Review comment: I have the same suggestions from `RemoteLogSegmentState` for this as well. Please refer to this comment: ########## File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java ########## @@ -0,0 +1,185 @@ +/* + * 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 { + log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata); + 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()); + } + + RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId(); + + RemoteLogMetadataCache remoteLogMetadataCache = partitionToRemoteLogMetadataCache + .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache()); + + remoteLogMetadataCache.addToInProgress(remoteLogSegmentMetadata); + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate) + throws RemoteStorageException { + log.debug("Updating remote log segment: [{}]", metadataUpdate); + Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null"); + + RemoteLogSegmentState targetState = metadataUpdate.state(); + // Callers should use putRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as Review comment: In the comment: s/putRemoteLogSegmentMetadata/addRemoteLogSegmentMetadata ########## File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java ########## @@ -0,0 +1,185 @@ +/* + * 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) Review comment: Really minor comment/discussion: Any reason to call this prefixed with `add` as `addRemoteLogSegmentMetadata` vs calling the deletion one prefixed with `put` as `putRemotePartitionDeleteMetadata` i.e. instead can these 2 methods both start with the same prefix either `add` or `put`? ---------------------------------------------------------------- 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