divijvaidya commented on code in PR #13947: URL: https://github.com/apache/kafka/pull/13947#discussion_r1293423318
########## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ########## @@ -343,21 +345,78 @@ public void onLeadershipChange(Set<Partition> partitionsBecomeLeader, /** * Deletes the internal topic partition info if delete flag is set as true. * - * @param topicPartition topic partition to be stopped. + * @param topicPartitions topic partitions that needs to be stopped. * @param delete flag to indicate whether the given topic partitions to be deleted or not. */ - public void stopPartitions(TopicPartition topicPartition, boolean delete) { + public void stopPartitions(Set<TopicPartition> topicPartitions, Review Comment: > Traversing the cache to remove those indexes will be expensive It's a 1000 entry cache. And we are traversing it only on topic deletion which happens rarely. Why do you think it will be expensive? Even if we change it to 10K entry cache, a for loop on a map against 10K entries is very cheap. On the other hand, the advantage is that we can make space for newer fetch calls immediately in the cache when we delete a very popular topic. ########## storage/api/src/test/java/org/apache/kafka/server/log/remote/storage/NoOpRemoteLogMetadataManager.java: ########## @@ -29,12 +29,16 @@ public class NoOpRemoteLogMetadataManager implements RemoteLogMetadataManager { @Override public CompletableFuture<Void> addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) { - return null; + CompletableFuture<Void> future = new CompletableFuture<>(); + future.complete(null); + return future; } @Override public CompletableFuture<Void> updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate remoteLogSegmentMetadataUpdate) { - return null; + CompletableFuture<Void> future = new CompletableFuture<>(); Review Comment: you can use https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html#completedFuture-U- -- 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