mumrah commented on code in PR #14612:
URL: https://github.com/apache/kafka/pull/14612#discussion_r1457908869


##########
core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala:
##########
@@ -140,6 +142,73 @@ class KRaftMetadataCache(val brokerId: Int) extends 
MetadataCache with Logging w
     }
   }
 
+  /**
+   * Return topic partition metadata for the given topic, listener and index 
range. Also, return the next partition
+   * index that is not included in the result.
+   *
+   * @param image                       The metadata image
+   * @param topicName                   The name of the topic.
+   * @param listenerName                The listener name.
+   * @param startIndex                  The smallest index of the partitions 
to be included in the result.
+   * @param upperIndex                  The upper limit of the index of the 
partitions to be included in the result.
+   *                                    Note that, the upper index can be 
larger than the largest partition index in
+   *                                    this topic.
+   * @return                            A collection of topic partition 
metadata and next partition index (-1 means
+   *                                    no next partition).
+   */
+  private def getPartitionMetadataForDescribeTopicResponse(
+    image: MetadataImage,
+    topicName: String,
+    listenerName: ListenerName,
+    startIndex: Int,
+    maxCount: Int
+  ): (Option[List[DescribeTopicPartitionsResponsePartition]], Int) = {
+    Option(image.topics().getTopic(topicName)) match {
+      case None => (None, -1)
+      case Some(topic) => {
+        val result = new ListBuffer[DescribeTopicPartitionsResponsePartition]()
+        val partitions = topic.partitions().keySet()
+        val upperIndex = topic.partitions().size().min(startIndex + maxCount)
+        val nextIndex = if (upperIndex < partitions.size()) upperIndex else -1
+        for (partitionId <- startIndex until upperIndex) {
+          topic.partitions().get(partitionId) match {
+            case partition : PartitionRegistration => {
+              val filteredReplicas = maybeFilterAliveReplicas(image, 
partition.replicas,
+                listenerName, false)
+              val filteredIsr = maybeFilterAliveReplicas(image, partition.isr, 
listenerName, false)
+              val offlineReplicas = getOfflineReplicas(image, partition, 
listenerName)
+              val maybeLeader = getAliveEndpoint(image, partition.leader, 
listenerName)
+              maybeLeader match {
+                case None =>
+                  result.append(new DescribeTopicPartitionsResponsePartition()
+                    .setPartitionIndex(partitionId)
+                    .setLeaderId(MetadataResponse.NO_LEADER_ID)
+                    .setLeaderEpoch(partition.leaderEpoch)
+                    .setReplicaNodes(filteredReplicas)
+                    .setIsrNodes(filteredIsr)
+                    .setOfflineReplicas(offlineReplicas)
+                    .setEligibleLeaderReplicas(Replicas.toList(partition.elr))
+                    .setLastKnownElr(Replicas.toList(partition.lastKnownElr)))
+                case Some(leader) =>
+                  result.append(new DescribeTopicPartitionsResponsePartition()
+                    .setPartitionIndex(partitionId)
+                    .setLeaderId(leader.id())
+                    .setLeaderEpoch(partition.leaderEpoch)
+                    .setReplicaNodes(filteredReplicas)
+                    .setIsrNodes(filteredIsr)
+                    .setOfflineReplicas(offlineReplicas)
+                    .setEligibleLeaderReplicas(Replicas.toList(partition.elr))
+                    .setLastKnownElr(Replicas.toList(partition.lastKnownElr)))
+              }
+            }
+            case _ =>

Review Comment:
   Should we throw an ISE here rather than silently continue? Maybe we could 
just log an error



-- 
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

Reply via email to