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/12/28 12:30:37 UTC

[GitHub] [pulsar] 315157973 opened a new pull request #13538: Add publishRateLimitedTimes to topic metrics

315157973 opened a new pull request #13538:
URL: https://github.com/apache/pulsar/pull/13538


   ### Motivation
   
   Broker limits the rate of publishing messages by setting `channel.setAutoRead(false)`. But no exception is returned to the client, or any log is printed, which makes it very difficult for user to troubleshoot the problem. When the Producer is limited, the client will send timeout and the connection will be closed (Broker no longer process the Client's heartbeat request, because autoRead=false). We don't know if there is a problem with network or the rate is limited or other reasons. 
   
   
   ### Modifications
   Add publishRateLimitedTimes to topic metrics
   
   
   ### Documentation
   - [x] `no-need-doc` 
   
   


-- 
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 change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -1710,7 +1710,10 @@ private void refreshTopicPublishRate() {
     public void checkBrokerPublishThrottlingRate() {
         brokerPublishRateLimiter.checkPublishRate();
         if (brokerPublishRateLimiter.isPublishRateExceeded()) {
-            forEachTopic(topic -> ((AbstractTopic) topic).disableProducerRead());
+            forEachTopic(topic -> {
+                topic.increasePublishLimitedTimes();

Review comment:
       This is duplicated count. Line 1715 will trigger `ServerCnx#disableCnxAutoRead`.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2496,6 +2497,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
             // When the quota of pending send requests is reached, stop reading from socket to cause backpressure on
             // client connection, possibly shared between multiple producers
             ctx.channel().config().setAutoRead(false);
+            recordRateLimitMetrics(producers.values());

Review comment:
       `producers.values()` makes a copy values.
   See `org.apache.pulsar.common.util.collections.ConcurrentLongHashMap#values`.




-- 
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 pull request #13538: Add publishRateLimitedTimes to topic metrics

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


   @315157973 Since the PR introduced a new metric, I added the doc-required label.


-- 
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 change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       > Each `ServerCnx` contains multiple Producers (all Producers that share this channel). I have traversed all the Producers here, and each Producer has its corresponding Topic, so all Topics that share this connection will be +1.
   
   Yes, that's why topic B will +1. 
   
   My scenario is that `startSendOperation` is only called in `ServerCnx1`. 
   `producers.values()` in Line 2481 only contains producers of Topic A and B (They shares the same connection `ServerCnx1` ).
   
   But `startSendOperation` is not called on  `ServerCnx2`, so the rate limit times of Topic C won't be +1, right?
   
   And in fact `ServerCnx2` is still rate limitted due to  this code path :
   `producer.getTopic().disableCnxAutoRead();` -->  `producers.values().forEach(producer -> producer.getCnx().disableCnxAutoRead());` (Topic A  has producer in ServerCnx2)
   
   




-- 
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] 315157973 commented on a change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       Each `ServerCnx` contains multiple Producers (all Producers that share this channel). I have traversed all the Producers here, and each Producer has its corresponding Topic, so all Topics that share this connection will be  +1.
   In the scenario you describe, the current limit times of Topic C will also be +1




-- 
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] 315157973 commented on a change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       `ServerCnx2` is another channel, connections bound to `ServerCnx1` will not effect connections bound to `ServerCnx2`. `producers.values()` only get the producers belong to `ServerCnx1`.
   
   So the code path you show will only disable auto read for connections bound to `ServerCnx1`.  Connections bound to `ServerCnx2` can go on producing.
   
   ```
   Connection (1) have Producer of topic A and B.
   Connection (2) have Producer of topic A and C.
   ```
   If Topic A is a multi-partition topic, only some partitions will be throttled. 
   




-- 
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 #13538: Add publishRateLimitedTimes to topic metrics

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


   


-- 
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 change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       > producer.getTopic().disableCnxAutoRead();
   
   This is going to throttle all the connections with producers of this `producer.getTopic()`. 
   We need to `recordRateLimitMetrics` on the topics in these connections too.
   
   For example.
   Connection (1) have Producer of topic A and B.
   Connection (2) have Producer of topic A and C.
   One send request in Connection (1) is causing topic A under throttling. 
   And Topic B and C is also under throttling because they share the same connection.
   And Topic C won't be called `increasePublishLimitedTimes` with current code.
   
   Correct me if I miss something.
   
   




-- 
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] 315157973 commented on a change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       `protected final ConcurrentHashMap<String, Producer> producers;`
   producers in AbstractTopic is not static, it only contains the producers belong to this  topic/partition
   Different topics will have different AbstractTopic objects




-- 
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] Anonymitaet commented on pull request #13538: Add publishRateLimitedTimes to topic metrics

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


   Discussed w/ @315157973, from the doc side, he will add the newly added metrics to https://pulsar.apache.org/docs/en/next/reference-metrics/#topic-metrics. 


-- 
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 change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       > `producers.values().forEach(producer -> producer.getCnx().disableCnxAutoRead());`
   
   This line is in `org.apache.pulsar.broker.service.AbstractTopic#disableCnxAutoRead`,  the `producers` is  `AbstractTopic#producers` not `ServerCnx#producers`.
   So it contains producers in both `ServerCnx1` and `ServerCnx2`




-- 
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] 315157973 commented on a change in pull request #13538: Add publishRateLimitedTimes to topic metrics

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



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -2477,6 +2478,7 @@ public void startSendOperation(Producer producer, int msgSize, int numMessages)
                     producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize);
             if (isPreciseTopicPublishRateExceeded) {
                 producer.getTopic().disableCnxAutoRead();
+                recordRateLimitMetrics(producers.values());

Review comment:
       Each `ServerCnx` contains multiple Producers (all Producers that share this channel). I have traversed all the Producers here, and each Producer has its corresponding Topic, so all Topics that share this connection will be  +1.
   In the scenario you describe, the rate limit times of Topic C will also be +1




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