apoorvmittal10 commented on code in PR #19666: URL: https://github.com/apache/kafka/pull/19666#discussion_r2085658297
########## core/src/main/java/kafka/server/share/SharePartitionCache.java: ########## @@ -0,0 +1,87 @@ +/* + * 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 kafka.server.share; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.server.share.SharePartitionKey; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +/** + * The SharePartitionCache is used to cache the SharePartition objects for each share group topic-partition. + * The cache is used to avoid creating new SharePartition instances. The cache is thread-safe. + */ +public class SharePartitionCache { + + /** + * The map to store the share group id and the set of topic-partitions for that group. + */ + private final Map<String, Set<TopicIdPartition>> groupMap; + /** + * The partition cache map is used to store the SharePartition objects for each share group topic-partition. + */ + private final Map<SharePartitionKey, SharePartition> partitionMap; + + SharePartitionCache() { + this.groupMap = new HashMap<>(); + this.partitionMap = new ConcurrentHashMap<>(); + } + + public SharePartition get(SharePartitionKey partitionKey) { + return partitionMap.get(partitionKey); + } + + public SharePartition put(SharePartitionKey partitionKey, SharePartition sharePartition) { + return partitionMap.put(partitionKey, sharePartition); + } + + public int size() { + return partitionMap.size(); + } + + public boolean containsKey(SharePartitionKey partitionKey) { + return partitionMap.containsKey(partitionKey); + } + + public boolean isEmpty() { + return partitionMap.isEmpty(); + } + + public synchronized SharePartition remove(SharePartitionKey partitionKey) { + groupMap.computeIfPresent(partitionKey.groupId(), (k, v) -> { + v.remove(partitionKey.topicIdPartition()); + return v; + }); + return partitionMap.remove(partitionKey); + } + + public synchronized SharePartition computeIfAbsent(SharePartitionKey partitionKey, Function<SharePartitionKey, SharePartition> mappingFunction) { + groupMap.putIfAbsent(partitionKey.groupId(), new HashSet<>()); + groupMap.get(partitionKey.groupId()).add(partitionKey.topicIdPartition()); + return partitionMap.computeIfAbsent(partitionKey, mappingFunction); + } + + public synchronized void removeGroup(String groupId) { + Set<TopicIdPartition> topicIdPartitions = groupMap.remove(groupId); Review Comment: Done. ########## core/src/main/java/kafka/server/share/SharePartitionManager.java: ########## @@ -651,14 +652,15 @@ void processShareFetch(ShareFetch shareFetch) { } private SharePartition getOrCreateSharePartition(SharePartitionKey sharePartitionKey) { - return partitionCacheMap.computeIfAbsent(sharePartitionKey, + return partitionCache.computeIfAbsent(sharePartitionKey, k -> { int leaderEpoch = ShareFetchUtils.leaderEpoch(replicaManager, sharePartitionKey.topicIdPartition().topicPartition()); // Attach listener to Partition which shall invoke partition change handlers. // However, as there could be multiple share partitions (per group name) for a single topic-partition, // hence create separate listeners per share partition which holds the share partition key // to identify the respective share partition. - SharePartitionListener listener = new SharePartitionListener(sharePartitionKey, replicaManager, partitionCacheMap); + SharePartitionListener listener = new SharePartitionListener(sharePartitionKey, replicaManager, 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. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org