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 2021/06/27 06:27:00 UTC

[GitHub] [pulsar] MarvinCai opened a new pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

MarvinCai opened a new pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119


   Fixes #https://github.com/apache/pulsar/issues/10495
   
   ### Motivation
   MsgDropRate info is missing after NonPersistentTopics admin api merged with Topics admin api. This PR is trying to fix this.
   
   ### Modifications
   Seems due to API merging, data is not properly deserialized in admin client.
   And also due to the added TopicsStats interface, the field hiding causing weird behavior with Jackson so fields in NonPersistentTopicStatsImpl intended to hide superclass' fields are not shown in output.
   
   Fixing by not using same field name to hide superclass fields and use @JsonIgnore to hide them from output. And add new fields to store subscription/publisher/replicator info for NonPersistentTopic.
   This does change the output name of those info, but it only changed in cli output, for admin client the old getSubscriptions/getSubscriptions/getReplication will still work. 
   ```
   {
     "count" : 0,
     "msgRateIn" : 0.0,
     "msgThroughputIn" : 0.0,
     "msgRateOut" : 0.0,
     "msgThroughputOut" : 0.0,
     "bytesInCounter" : 0,
     "msgInCounter" : 0,
     "bytesOutCounter" : 0,
     "msgOutCounter" : 0,
     "averageMsgSize" : 0.0,
     "msgChunkPublished" : false,
     "storageSize" : 0,
     "backlogSize" : 0,
     "offloadedStorageSize" : 0,
     "waitingPublishers" : 0,
     "nonContiguousDeletedMessagesRanges" : 0,
     "nonContiguousDeletedMessagesRangesSerializedSize" : 0,
     "msgDropRate" : 0.0,
     "nonPersistentPublishers" : [ ],
     "nonPersistentSubscriptions" : {
       "my-sub" : {
         "msgRateOut" : 0.0,
         "msgThroughputOut" : 0.0,
         "bytesOutCounter" : 0,
         "msgOutCounter" : 0,
         "msgRateRedeliver" : 0.0,
         "chunkedMessageRate" : 0,
         "msgBacklog" : 0,
         "backlogSize" : 0,
         "msgBacklogNoDelayed" : 0,
         "blockedSubscriptionOnUnackedMsgs" : false,
         "msgDelayed" : 0,
         "unackedMessages" : 0,
         "type" : "Exclusive",
         "msgRateExpired" : 0.0,
         "totalMsgExpired" : 0,
         "lastExpireTimestamp" : 0,
         "lastConsumedFlowTimestamp" : 0,
         "lastConsumedTimestamp" : 0,
         "lastAckedTimestamp" : 0,
         "lastMarkDeleteAdvancedTimestamp" : 0,
         "consumers" : [ {
           "msgRateOut" : 0.0,
           "msgThroughputOut" : 0.0,
           "bytesOutCounter" : 0,
           "msgOutCounter" : 0,
           "msgRateRedeliver" : 0.0,
           "chunkedMessageRate" : 0.0,
           "consumerName" : "26c56",
           "availablePermits" : 1000,
           "unackedMessages" : 0,
           "avgMessagesPerEntry" : 1000,
           "blockedConsumerOnUnackedMsgs" : false,
           "lastAckedTimestamp" : 0,
           "lastConsumedTimestamp" : 0,
           "metadata" : { },
           "address" : "/127.0.0.1:52121",
           "connectedSince" : "2021-06-27T14:20:43.709766+08:00",
           "clientVersion" : "2.9.0-SNAPSHOT"
         } ],
         "isDurable" : false,
         "isReplicated" : false,
         "consumersAfterMarkDeletePosition" : { },
         "nonContiguousDeletedMessagesRanges" : 0,
         "nonContiguousDeletedMessagesRangesSerializedSize" : 0,
         "msgDropRate" : 0.0,
         "durable" : false,
         "replicated" : false
       }
     },
     "nonPersistentReplicators" : { }
   }
   ```
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   This change added tests and can be verified as follows:
   - Added unit test.
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API: yes
     - The schema: no
     - The default values of configurations: no
     - The wire protocol: no
     - The rest endpoints: no
     - The admin cli options: no
     - Anything that affects deployment: no
   


-- 
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] MarvinCai commented on pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#issuecomment-876338982


   /pulsarbot rerun-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] MarvinCai commented on pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#issuecomment-876447466


   @codelipenghui addressed your comments, 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] MarvinCai commented on a change in pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on a change in pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#discussion_r661146025



##########
File path: pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java
##########
@@ -849,6 +850,62 @@ default TopicStats getStats(String topic) throws PulsarAdminException {
         return getStatsAsync(topic, false, false);
     }
 
+
+    /**
+     * Get the stats for non-persistent topic.
+     * All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
+     *
+     * @param topic
+     *            topic name
+     * @param getPreciseBacklog
+     *            Set to true to get precise backlog, Otherwise get imprecise backlog.
+     * @param subscriptionBacklogSize
+     *            Whether to get backlog size for each subscription.
+     * @return the topic statistics
+     *
+     * @throws NotAuthorizedException
+     *             Don't have admin permission
+     * @throws NotFoundException
+     *             Topic does not exist
+     * @throws PulsarAdminException
+     *             Unexpected error
+     */
+    NonPersistentTopicStats getStatsNonPersistent(String topic, boolean getPreciseBacklog,

Review comment:
       @codelipenghui 
   I've tried that before, once the response object got serialized as TopicStats, even we convert it to NonPersistentTopicStats later the msgDropRate field will be missing, I think it's just lost during deserialization, so seems we'll have to deserialize to the desired type or interface when we get the response(NonPersistentTopicStats).




-- 
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] MarvinCai commented on a change in pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on a change in pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#discussion_r661171413



##########
File path: pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java
##########
@@ -849,6 +850,62 @@ default TopicStats getStats(String topic) throws PulsarAdminException {
         return getStatsAsync(topic, false, false);
     }
 
+
+    /**
+     * Get the stats for non-persistent topic.
+     * All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
+     *
+     * @param topic
+     *            topic name
+     * @param getPreciseBacklog
+     *            Set to true to get precise backlog, Otherwise get imprecise backlog.
+     * @param subscriptionBacklogSize
+     *            Whether to get backlog size for each subscription.
+     * @return the topic statistics
+     *
+     * @throws NotAuthorizedException
+     *             Don't have admin permission
+     * @throws NotFoundException
+     *             Topic does not exist
+     * @throws PulsarAdminException
+     *             Unexpected error
+     */
+    NonPersistentTopicStats getStatsNonPersistent(String topic, boolean getPreciseBacklog,

Review comment:
       @codelipenghui 
   I just tried that, once the response object got deserialized as TopicStats, it's actual type will be TopicStatsImpl at client side(we set the binding at ObjectMapperFactory [ref](https://github.com/apache/pulsar/blob/3cbfd7454ce824a720d1dc645ea23ee3831ffff0/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java#L179)), even server send it out as NonPersistentTopicStatsImpl, and we won't be able to convert the object back to NonPersistentTopicStats or NonPersistentTopicStatsImpl. 
   I think we'll have to deserialize the response to the desired type when we receive it. So a new method is needed for this purpose.




-- 
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] codelipenghui commented on a change in pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#discussion_r681392477



##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java
##########
@@ -47,31 +45,49 @@
     @Getter
     public double msgDropRate;
 
-    /** List of connected publishers on this topic w/ their stats. */
-    @Getter
-    public List<? extends NonPersistentPublisherStats> publishers;
+    @JsonIgnore
+    public List<PublisherStatsImpl> publishers;
 
-    /** Map of subscriptions with their individual statistics. */
-    @Getter
-    public Map<String, ? extends NonPersistentSubscriptionStats> subscriptions;
+    @JsonIgnore
+    public Map<String, SubscriptionStatsImpl> subscriptions;
 
-    /** Map of replication statistics by remote cluster context. */
-    @Getter
-    public Map<String, ? extends NonPersistentReplicatorStats> replication;
+    @JsonIgnore
+    public Map<String, ReplicatorStatsImpl> replication;
+
+    public List<NonPersistentPublisherStats> getNonPersistentPublishers() {
+        return (List<NonPersistentPublisherStats>) nonPersistentPublishers;
+    }
+
+    public Map<String, NonPersistentSubscriptionStats> getNonPersistentSubscriptions() {
+        return (Map<String, NonPersistentSubscriptionStats>) nonPersistentSubscriptions;
+    }
+
+    public Map<String, NonPersistentReplicatorStats> getNonPersistentReplicators() {
+        return (Map<String, NonPersistentReplicatorStats>) nonPersistentReplicators;
+    }
+
+    /** List of connected publishers on this non-persistent topic w/ their stats. */
+    public List<? extends NonPersistentPublisherStats> nonPersistentPublishers;
+
+    /** Map of non-persistent subscriptions with their individual statistics. */
+    public Map<String, ? extends NonPersistentSubscriptionStats> nonPersistentSubscriptions;
+
+    /** Map of non-persistent replication statistics by remote cluster context. */
+    public Map<String, ? extends NonPersistentReplicatorStats> nonPersistentReplicators;

Review comment:
       Can we add @JsonAlias to make sure the name that exposed to the JSON result not changed?




-- 
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] sijie commented on pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
sijie commented on pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#issuecomment-885911304


   @codelipenghui Can you review this?


-- 
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] codelipenghui commented on a change in pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#discussion_r660599308



##########
File path: pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java
##########
@@ -849,6 +850,62 @@ default TopicStats getStats(String topic) throws PulsarAdminException {
         return getStatsAsync(topic, false, false);
     }
 
+
+    /**
+     * Get the stats for non-persistent topic.
+     * All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
+     *
+     * @param topic
+     *            topic name
+     * @param getPreciseBacklog
+     *            Set to true to get precise backlog, Otherwise get imprecise backlog.
+     * @param subscriptionBacklogSize
+     *            Whether to get backlog size for each subscription.
+     * @return the topic statistics
+     *
+     * @throws NotAuthorizedException
+     *             Don't have admin permission
+     * @throws NotFoundException
+     *             Topic does not exist
+     * @throws PulsarAdminException
+     *             Unexpected error
+     */
+    NonPersistentTopicStats getStatsNonPersistent(String topic, boolean getPreciseBacklog,

Review comment:
       The NonPersistentTopicStats also implement the TopicStats, So I think we do not need to introduce a new method in the API? We can just use the `getStats` method, and if users want to get the msgDropRate, they can convert TopicStats to NonPersistentTopicStats directly?




-- 
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] MarvinCai commented on a change in pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on a change in pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119#discussion_r684724602



##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java
##########
@@ -47,31 +45,49 @@
     @Getter
     public double msgDropRate;
 
-    /** List of connected publishers on this topic w/ their stats. */
-    @Getter
-    public List<? extends NonPersistentPublisherStats> publishers;
+    @JsonIgnore
+    public List<PublisherStatsImpl> publishers;
 
-    /** Map of subscriptions with their individual statistics. */
-    @Getter
-    public Map<String, ? extends NonPersistentSubscriptionStats> subscriptions;
+    @JsonIgnore
+    public Map<String, SubscriptionStatsImpl> subscriptions;
 
-    /** Map of replication statistics by remote cluster context. */
-    @Getter
-    public Map<String, ? extends NonPersistentReplicatorStats> replication;
+    @JsonIgnore
+    public Map<String, ReplicatorStatsImpl> replication;
+
+    public List<NonPersistentPublisherStats> getNonPersistentPublishers() {
+        return (List<NonPersistentPublisherStats>) nonPersistentPublishers;
+    }
+
+    public Map<String, NonPersistentSubscriptionStats> getNonPersistentSubscriptions() {
+        return (Map<String, NonPersistentSubscriptionStats>) nonPersistentSubscriptions;
+    }
+
+    public Map<String, NonPersistentReplicatorStats> getNonPersistentReplicators() {
+        return (Map<String, NonPersistentReplicatorStats>) nonPersistentReplicators;
+    }
+
+    /** List of connected publishers on this non-persistent topic w/ their stats. */
+    public List<? extends NonPersistentPublisherStats> nonPersistentPublishers;
+
+    /** Map of non-persistent subscriptions with their individual statistics. */
+    public Map<String, ? extends NonPersistentSubscriptionStats> nonPersistentSubscriptions;
+
+    /** Map of non-persistent replication statistics by remote cluster context. */
+    public Map<String, ? extends NonPersistentReplicatorStats> nonPersistentReplicators;

Review comment:
       @JsonAlias seems only used during deserialization, used @JasonProperty to keep the output field name. Updated issue description.




-- 
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] codelipenghui merged pull request #11119: Fix MsgDropRate missing from NonPersistentTopics stats output.

Posted by GitBox <gi...@apache.org>.
codelipenghui merged pull request #11119:
URL: https://github.com/apache/pulsar/pull/11119


   


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