Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

topic publish rate limit not take effect #6975

Closed
aloyszhang opened this issue May 18, 2020 · 4 comments · Fixed by #7078
Closed

topic publish rate limit not take effect #6975

aloyszhang opened this issue May 18, 2020 · 4 comments · Fixed by #7078
Labels
area/broker help wanted type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages

Comments

@aloyszhang
Copy link
Contributor

aloyszhang commented May 18, 2020

Describe the bug
set-publish-rate does not take effect

To Reproduce

  1. set-publish-rate
  2. do publish

Expected behavior
producer should not publish messages faster than the publish rate

Screenshots
publish rate limit information

bin/pulsar-admin namespaces get-publish-rate rate/rate 
{
  "publishThrottlingRateInMsg" : 10,
  "publishThrottlingRateInByte" : 10240
}

do publish

bin/pulsar-perf produce -threads 1 -u pulsar://100.76.43.216:6650 -o 10000 -n 1 -b 0 -bm 0 -s 1024 -r 100000 rate/rate/rate

14:43:32.193 [main] INFO  org.apache.pulsar.testclient.PerformanceProducer - Throughput produced:    161.6  msg/s ---      1.4 Mbit/s --- failure      0.0 msg/s --- Latency: mean: 5314.773 ms - med: 5730.591 - 95pct: 8716.543 - 99pct: 8744.191 - 99.9pct: 8744.319 - 99.99pct: 8744.319 - Max: 8744.319
14:43:42.262 [main] INFO  org.apache.pulsar.testclient.PerformanceProducer - Throughput produced:    214.2  msg/s ---      1.8 Mbit/s --- failure      0.0 msg/s --- Latency: mean: 14039.659 ms - med: 13728.319 - 95pct: 18667.391 - 99pct: 18700.927 - 99.9pct: 18701.055 - 99.99pct: 18701.055 - Max: 18701.055

As shown, under the limit of 10 msgs and 10240 bytes, when send message with 1024B bytes, it should not be faster than 10msg/s.
But the tes result is much more than 10msg/s.
Additional context
As my researching , pulsar do topic publish rate limit using two individual thread,


 long topicTickTimeMs = pulsar().getConfiguration().getTopicPublisherThrottlingTickTimeMillis();
        if (topicTickTimeMs > 0) {
            if (this.topicPublishRateLimiterMonitor == null) {
                this.topicPublishRateLimiterMonitor = Executors.newSingleThreadScheduledExecutor(
                        new DefaultThreadFactory("pulsar-topic-publish-rate-limiter-monitor"));
                if (topicTickTimeMs > 0) {
                    topicPublishRateLimiterMonitor.scheduleAtFixedRate(safeRun(() -> checkTopicPublishThrottlingRate()),
                            topicTickTimeMs, topicTickTimeMs, TimeUnit.MILLISECONDS);
                    
                    topicPublishRateLimiterMonitor.scheduleAtFixedRate(safeRun(() -> refreshTopicPublishRate()), 1, 1,
                            TimeUnit.SECONDS);
                }
            }
        } 

one thread schedule task that sums up publish-rate across all cnx on a topic and another schedule task that refreshes rate-limiting bucket.
This means we can only pause publish after send message for topicPublisherThrottlingTickTimeMillis long times, and messages send before topicPublisherThrottlingTickTimeMillis will never trigger rate limit.

In order to get efficient publish rate limit, I think, we should use something like RateLimiter instead of a period task.

@codelipenghui
Copy link
Contributor

@aloyszhang Thanks for the feedback, I think this is an enhancement. I would like to change the tag to enhancement.

@codelipenghui codelipenghui added type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages and removed type/bug The PR fixed a bug or issue reported a bug labels May 18, 2020
@aloyszhang
Copy link
Contributor Author

@codelipenghui Thanks for your suggestion, LGTM.

@codelipenghui
Copy link
Contributor

@aloyszhang I think we can add options in the broker.conf. So that users can choose the precise rate control or not. If users don't care about the cost of the competition, they can choose the precise rate control.

@aloyszhang
Copy link
Contributor Author

@codelipenghui OK, I'll add the option.

aloyszhang added a commit to aloyszhang/pulsar that referenced this issue May 28, 2020
aloyszhang added a commit to aloyszhang/pulsar that referenced this issue May 28, 2020
codelipenghui pushed a commit that referenced this issue Jun 2, 2020
Fixes #6975 
### Motivation

  For now,  pulsar limits  publish rate of topic by a period task runs every `topicPublisherThrottlingTickTimeMillis`. That means in the time `topicPublisherThrottlingTickTimeMillis`, the limit can't take effect. 
  This PR enable precise topic publish rate limit on broker.
Huanli-Meng pushed a commit to Huanli-Meng/pulsar that referenced this issue Jun 12, 2020
Fixes apache#6975 
### Motivation

  For now,  pulsar limits  publish rate of topic by a period task runs every `topicPublisherThrottlingTickTimeMillis`. That means in the time `topicPublisherThrottlingTickTimeMillis`, the limit can't take effect. 
  This PR enable precise topic publish rate limit on broker.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this issue Aug 24, 2020
Fixes apache#6975 
### Motivation

  For now,  pulsar limits  publish rate of topic by a period task runs every `topicPublisherThrottlingTickTimeMillis`. That means in the time `topicPublisherThrottlingTickTimeMillis`, the limit can't take effect. 
  This PR enable precise topic publish rate limit on broker.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker help wanted type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants