From 85a0f79bbf3cadeb9a3ebf58fdb93464e5eefa1d Mon Sep 17 00:00:00 2001
From: Zixuan Liu <nodeces@gmail.com>
Date: Mon, 13 Jan 2025 10:39:02 +0800
Subject: [PATCH] [fix][broker] Remove blocking calls from
 internalGetPartitionedStats (#23832)

Signed-off-by: Zixuan Liu <nodeces@gmail.com>
(cherry picked from commit dc6397093eb645fec1bfd6f127f78ad36be0a360)
---
 .../admin/impl/PersistentTopicsBase.java      | 30 +++++++++----------
 1 file changed, 14 insertions(+), 16 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
index 1cc93ad5ba885..08fe12178a85b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
@@ -1597,23 +1597,21 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                     }
                 }
                 if (perPartition && stats.partitions.isEmpty()) {
-                    try {
-                        boolean pathExists = namespaceResources().getPartitionedTopicResources()
-                                .partitionedTopicExists(topicName);
-                        if (pathExists) {
-                            stats.partitions.put(topicName.toString(), new TopicStatsImpl());
-                        } else {
-                            asyncResponse.resume(
-                                    new RestException(Status.NOT_FOUND,
-                                            "Internal topics have not been generated yet"));
-                            return null;
-                        }
-                    } catch (Exception e) {
-                        asyncResponse.resume(new RestException(e));
-                        return null;
-                    }
+                    namespaceResources().getPartitionedTopicResources()
+                            .partitionedTopicExistsAsync(topicName)
+                            .thenAccept(exists -> {
+                                if (exists) {
+                                    stats.partitions.put(topicName.toString(), new TopicStatsImpl());
+                                    asyncResponse.resume(stats);
+                                } else {
+                                    asyncResponse.resume(
+                                            new RestException(Status.NOT_FOUND,
+                                                    "Internal topics have not been generated yet"));
+                                }
+                            });
+                } else {
+                    asyncResponse.resume(stats);
                 }
-                asyncResponse.resume(stats);
                 return null;
             });
         }).exceptionally(ex -> {