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


##########
core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala:
##########
@@ -155,72 +153,78 @@ class KRaftMetadataCache(val brokerId: Int) extends 
MetadataCache with Logging w
    * @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 whether there are more partitions.
+   * @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,
-    upperIndex: Int
-  ): (Option[List[DescribeTopicPartitionsResponsePartition]], Boolean) = {
+    maxCount: Int
+  ): (Option[List[DescribeTopicPartitionsResponsePartition]], Int) = {
     Option(image.topics().getTopic(topicName)) match {
-      case None => (None, false)
+      case None => (None, -1)
       case Some(topic) => {
         val result = new ListBuffer[DescribeTopicPartitionsResponsePartition]()
-        val endIndex = upperIndex.min(topic.partitions().size())
-        for (partitionId <- startIndex until endIndex) {
-          val partition = topic.partitions().get(partitionId)
-          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 =>
-              val error = if 
(!image.cluster().brokers.containsKey(partition.leader)) {
-                debug(s"Error while fetching metadata for 
$topicName-$partitionId: leader not available")
-                Errors.LEADER_NOT_AVAILABLE
-              } else {
-                debug(s"Error while fetching metadata for 
$topicName-$partitionId: listener $listenerName " +
-                  s"not found on leader ${partition.leader}")
-                Errors.LISTENER_NOT_FOUND
-              }
-              result.addOne(new DescribeTopicPartitionsResponsePartition()
-                .setErrorCode(error.code)
-                .setPartitionIndex(partitionId)
-                .setLeaderId(MetadataResponse.NO_LEADER_ID)
-                .setLeaderEpoch(partition.leaderEpoch)
-                .setReplicaNodes(filteredReplicas)
-                .setIsrNodes(filteredIsr)
-                .setOfflineReplicas(offlineReplicas))
-            case Some(leader) =>
-              val error = if (filteredReplicas.size < 
partition.replicas.length) {
-                debug(s"Error while fetching metadata for 
$topicName-$partitionId: replica information not available for " +
-                  s"following brokers 
${partition.replicas.filterNot(filteredReplicas.contains).mkString(",")}")
-                Errors.REPLICA_NOT_AVAILABLE
-              } else if (filteredIsr.size < partition.isr.length) {
-                debug(s"Error while fetching metadata for 
$topicName-$partitionId: in sync replica information not available for " +
-                  s"following brokers 
${partition.isr.filterNot(filteredIsr.contains).mkString(",")}")
-                Errors.REPLICA_NOT_AVAILABLE
-              } else {
-                Errors.NONE
-              }
-
-              result.addOne(new DescribeTopicPartitionsResponsePartition()
-                .setErrorCode(error.code)
-                .setPartitionIndex(partitionId)
-                .setLeaderId(leader.id())
-                .setLeaderEpoch(partition.leaderEpoch)
-                .setReplicaNodes(filteredReplicas)
-                .setIsrNodes(filteredIsr)
-                .setOfflineReplicas(offlineReplicas)
-                .setEligibleLeaderReplicas(Replicas.toList(partition.elr))
-                .setLastKnownElr(Replicas.toList(partition.lastKnownElr)))
+        // The partition id may not be consecutive.
+        val partitions = 
topic.partitions().keySet().stream().sorted().iterator()

Review Comment:
   This has O(N*logN) runtime complexity and O(N) space complexity.  We could 
do O(N) complexity and not have an extra copy if we just iterate over all 
partitions and filter the ones that fit into the required range (one of your 
previous implementations had this).



##########
core/src/main/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandler.java:
##########
@@ -103,14 +102,20 @@ public DescribeTopicPartitionsResponseData 
handleDescribeTopicPartitionsRequest(
                 );
             }
             return isAuthorized;
-        }).sorted().iterator();
+        }).sorted();
+
+        // Reset the first partition index if the cursor topic is missing from 
the authorized topic list.
+        int firstPartitionId = !cursorTopicName.isEmpty() && 
authHelper.authorize(
+            abstractRequest.context(), DESCRIBE, TOPIC, cursorTopicName, true, 
true, 1)
+            ? cursor.partitionIndex() : 0;

Review Comment:
   We'd need to handle both non-authorized and deleted cursor topic.  If we 
could peek if the first item in the authorized stream is equal to cursor topic, 
we can handle both.
   
   If we follow @mumrah's suggestion, we wouldn't have the sorted stream 
anymore, in that case, we should check if partitionIndex >0 && 
!currentTopic.equals(cursorTopic) then use partitionIndex 0 (in other words, 
non-0 partition index can only be applied if we are processing cursor topic 
(which would be the first topic, unless it gets deleted or unauthorized).



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