You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/08/02 10:04:02 UTC

[GitHub] [pulsar] AnonHxy opened a new pull request, #16916: [Improve][Broker]Reduce PartitionedStats local REST call

AnonHxy opened a new pull request, #16916:
URL: https://github.com/apache/pulsar/pull/16916

   ### Motivation
   
   
   * Reduce local REST calll for `org.apache.pulsar.broker.admin.impl.PersistentTopicsBase#internalGetPartitionedStats`
   
   ### Modifications
   
   * Call local method if partition is owned by this broker,  or call REST api usng internal `AdminClient`
   
   ### Verifying this change
   
   - [x] Make sure that the change passes the CI checks.
   
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [x] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1204113514

   /pulsarbot CI - Build - MacOS / Unit-BROKER_FLAKY Tests (pull_request)


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1205078911

   @HQebupt   @codelipenghui @Jason918  PTAL ~


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on a diff in pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on code in PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#discussion_r935634969


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1342,16 +1343,35 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                 return;
             }
             PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata);
-            List<CompletableFuture<TopicStats>> topicStatsFutureList = Lists.newArrayList();
+            List<CompletableFuture<TopicStats>> topicStatsFutureList = new ArrayList<>(partitionMetadata.partitions);
+            AtomicBoolean exceptionFlag = new AtomicBoolean(false);

Review Comment:
   Nice catch. I get your point.  Have removed the `exceptionFlag`



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on a diff in pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on code in PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#discussion_r935495233


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1342,16 +1343,35 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                 return;
             }
             PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata);
-            List<CompletableFuture<TopicStats>> topicStatsFutureList = Lists.newArrayList();
+            List<CompletableFuture<TopicStats>> topicStatsFutureList = new ArrayList<>(partitionMetadata.partitions);
+            AtomicBoolean exceptionFlag = new AtomicBoolean(false);

Review Comment:
   This check will break the loop if  throws `PulsarServerException`, that will avoid querying the left partitions stats. So I think it is useful @Technoboy-  



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1206157088

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Jason918 commented on a diff in pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#discussion_r937648861


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1342,17 +1342,31 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                 return;
             }
             PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata);
-            List<CompletableFuture<TopicStats>> topicStatsFutureList = Lists.newArrayList();
+            List<CompletableFuture<TopicStats>> topicStatsFutureList = new ArrayList<>(partitionMetadata.partitions);
             for (int i = 0; i < partitionMetadata.partitions; i++) {
-                try {
-                    topicStatsFutureList
-                            .add(pulsar().getAdminClient().topics().getStatsAsync(
-                                    (topicName.getPartition(i).toString()), getPreciseBacklog, subscriptionBacklogSize,
-                                    getEarliestTimeInBacklog));
-                } catch (PulsarServerException e) {
-                    asyncResponse.resume(new RestException(e));
-                    return;
-                }
+                TopicName partition = topicName.getPartition(i);
+                topicStatsFutureList.add(
+                    pulsar().getNamespaceService()
+                        .isServiceUnitOwnedAsync(partition)
+                        .thenCompose(owned -> {
+                            if (owned) {
+                                return getTopicReferenceAsync(partition)
+                                    .thenApply(ref ->
+                                        ref.getStats(getPreciseBacklog, subscriptionBacklogSize,
+                                            getEarliestTimeInBacklog));
+                            } else {
+                                CompletableFuture<TopicStats> restFuture = new CompletableFuture<>();
+                                try {
+                                    restFuture = pulsar().getAdminClient().topics().getStatsAsync(
+                                        partition.toString(), getPreciseBacklog, subscriptionBacklogSize,
+                                        getEarliestTimeInBacklog);
+                                } catch (PulsarServerException e) {
+                                    restFuture.completeExceptionally(e);
+                                }
+                                return restFuture;

Review Comment:
   ```suggestion
                                   try {
                                       return pulsar().getAdminClient().topics().getStatsAsync(
                                           partition.toString(), getPreciseBacklog, subscriptionBacklogSize,
                                           getEarliestTimeInBacklog);
                                   } catch (PulsarServerException e) {
                                     org.apache.pulsar.common.util.FutureUtil#failedFuture(e);
                                   }
   ```
   Reduce a CompletableFuture object if no exception.



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1207354930

   @AnonHxy Please provide a correct documentation label for your PR.
   Instructions see [Pulsar Documentation Label Guide](https://docs.google.com/document/d/1Qw7LHQdXWBW9t2-r-A7QdFDBwmZh6ytB4guwMoXHqc0).


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1203408929

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- merged pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
Technoboy- merged PR #16916:
URL: https://github.com/apache/pulsar/pull/16916


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1203508444

   /pulsarbot run-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] AnonHxy commented on a diff in pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on code in PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#discussion_r937845273


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1342,17 +1342,31 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                 return;
             }
             PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata);
-            List<CompletableFuture<TopicStats>> topicStatsFutureList = Lists.newArrayList();
+            List<CompletableFuture<TopicStats>> topicStatsFutureList = new ArrayList<>(partitionMetadata.partitions);
             for (int i = 0; i < partitionMetadata.partitions; i++) {
-                try {
-                    topicStatsFutureList
-                            .add(pulsar().getAdminClient().topics().getStatsAsync(
-                                    (topicName.getPartition(i).toString()), getPreciseBacklog, subscriptionBacklogSize,
-                                    getEarliestTimeInBacklog));
-                } catch (PulsarServerException e) {
-                    asyncResponse.resume(new RestException(e));
-                    return;
-                }
+                TopicName partition = topicName.getPartition(i);
+                topicStatsFutureList.add(
+                    pulsar().getNamespaceService()
+                        .isServiceUnitOwnedAsync(partition)
+                        .thenCompose(owned -> {
+                            if (owned) {
+                                return getTopicReferenceAsync(partition)
+                                    .thenApply(ref ->
+                                        ref.getStats(getPreciseBacklog, subscriptionBacklogSize,
+                                            getEarliestTimeInBacklog));
+                            } else {
+                                CompletableFuture<TopicStats> restFuture = new CompletableFuture<>();
+                                try {
+                                    restFuture = pulsar().getAdminClient().topics().getStatsAsync(
+                                        partition.toString(), getPreciseBacklog, subscriptionBacklogSize,
+                                        getEarliestTimeInBacklog);
+                                } catch (PulsarServerException e) {
+                                    restFuture.completeExceptionally(e);
+                                }
+                                return restFuture;

Review Comment:
   Nice suggestion. Have updated @Jason918 



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on a diff in pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#discussion_r935448235


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1342,16 +1343,35 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                 return;
             }
             PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata);
-            List<CompletableFuture<TopicStats>> topicStatsFutureList = Lists.newArrayList();
+            List<CompletableFuture<TopicStats>> topicStatsFutureList = new ArrayList<>(partitionMetadata.partitions);
+            AtomicBoolean exceptionFlag = new AtomicBoolean(false);

Review Comment:
   Maybe we can remove this line and check `exception` at line 1378



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on a diff in pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#discussion_r935581296


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1342,16 +1343,35 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
                 return;
             }
             PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata);
-            List<CompletableFuture<TopicStats>> topicStatsFutureList = Lists.newArrayList();
+            List<CompletableFuture<TopicStats>> topicStatsFutureList = new ArrayList<>(partitionMetadata.partitions);
+            AtomicBoolean exceptionFlag = new AtomicBoolean(false);

Review Comment:
   `PulsarServerException` throw by `pulsar().getAdminClient()`. If this occurs, there will be a fast failed result.



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on pull request #16916: [Improve][Broker]Reduce PartitionedStats local REST call

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #16916:
URL: https://github.com/apache/pulsar/pull/16916#issuecomment-1207354850

   @AnonHxy Please provide a correct documentation label for your PR.
   Instructions see [Pulsar Documentation Label Guide](https://docs.google.com/document/d/1Qw7LHQdXWBW9t2-r-A7QdFDBwmZh6ytB4guwMoXHqc0).


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org