kamalcph commented on code in PR #20256: URL: https://github.com/apache/kafka/pull/20256#discussion_r2307889453
########## storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java: ########## @@ -490,8 +479,8 @@ public void onLeadershipChange(Set<TopicPartitionLog> partitionsBecomeLeader, public void stopLeaderCopyRLMTasks(Set<TopicPartitionLog> partitions) { for (TopicPartitionLog partition : partitions) { TopicPartition tp = partition.topicPartition(); - if (topicIdByPartitionMap.containsKey(tp)) { - TopicIdPartition tpId = new TopicIdPartition(topicIdByPartitionMap.get(tp), tp); + if (metadataCache.contains(tp)) { Review Comment: This `if` check can be removed once TopicIdPartition gets introduced in the TopicPartitionLog. This can be taken up in the next / separate PR. ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -2347,6 +2347,7 @@ class ReplicaManager(val config: KafkaConfig, // Before taking the lock, compute the local changes val localChanges = delta.localChanges(config.nodeId) val metadataVersion = newImage.features().metadataVersionOrThrow() + val deletedTopicIdMap = delta.image().topicNameToIdView() Review Comment: nit: Can this be renamed to `deletedTopicIdMap` -> `topicNamesToIds` for clarity? ########## core/src/main/scala/kafka/server/ConfigHandler.scala: ########## @@ -90,12 +90,14 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, val stopPartitions: java.util.HashSet[StopPartition] = new java.util.HashSet[StopPartition]() leaderPartitions.foreach(partition => { // delete remote logs and stop RemoteLogMetadataManager - stopPartitions.add(new StopPartition(partition.topicPartition, false, true, true)) + stopPartitions.add( + new StopPartition(new TopicIdPartition(partition.topicId.get, partition.topicPartition), false, true, true)) Review Comment: To rule out empty topicId, can we take the topicId from the metadataCache itself? ``` val topicId = replicaManager.metadataCache.getTopicId(partition.topic) ``` -- 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