Skip to content

Conversation

@codelipenghui
Copy link
Contributor

Motivation

The consumer will apply the default batch receive policy even if the user will not use the batch receive API.

public static final BatchReceivePolicy DEFAULT_POLICY = new BatchReceivePolicy(
-1, 10 * 1024 * 1024, 100, TimeUnit.MILLISECONDS);

This will consume lots of CPU if the client has many consumers (100k consumers)

image

consumer-cpu-threads.html.txt

The Pulsar perf tool can also reproduce the problem if run the test with many consumers

Modification

If there is no pending batch receive operation for a consumer, no need to trigger the
batch timeout task periodically. We can only start the timeout check after adding batch
receive request to pending request queue.

Remove the lock in MultiTopicsConsumerImpl as #10352 does

Verification

Added new test to verify the batch receive timeout task will not start if no batch
receive request

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)

  • doc-not-needed
    (Please explain why)

  • doc
    (Your PR contains doc changes)

  • doc-complete
    (Docs have been already added)

…ving pending batch receives requests

### Motivation

The consumer will apply the default batch receive policy even if the user will not use the batch receive API.

https://github.com/apache/pulsar/blob/6704f12104219611164aa2bb5bbdfc929613f1bf/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java#L60-L61

This will consume lots of CPU if the client have many consumers (100k consumers)

The Pulsar perf tool can also reproduce the problem if run the test with many consumers

### Modification

If there is no pending batch receive operation for a consumer, no need to trigger the
batch timeout task periodically. We can only start the timeout check after adding batch
receive request to pending request queue.

Remove the lock in MultiTopicsConsumerImpl as apache#10352 does

### Verification

Added new test to verify the batch receive timeout task will not start if no batch
receive request
@codelipenghui codelipenghui self-assigned this Jun 21, 2022
@codelipenghui codelipenghui added this to the 2.11.0 milestone Jun 21, 2022
@codelipenghui codelipenghui added type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages component/client-java release/2.10.2 release/2.9.4 labels Jun 21, 2022
@github-actions
Copy link

@codelipenghui Please provide a correct documentation label for your PR.
Instructions see Pulsar Documentation Label Guide.

@codelipenghui codelipenghui added doc-not-needed Your PR changes do not impact docs and removed doc-label-missing labels Jun 21, 2022
Copy link
Member

@michaeljmarshall michaeljmarshall left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@codelipenghui - this is a great improvement. I have one concern about concurrent timer tasks, but otherwise, this looks good.

Copy link
Member

@michaeljmarshall michaeljmarshall left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@codelipenghui codelipenghui merged commit a0ccdc9 into apache:master Jun 23, 2022
@codelipenghui codelipenghui deleted the penghui/optimize_batch_receive_timeout branch June 23, 2022 01:04
codelipenghui added a commit that referenced this pull request Jun 28, 2022
…ving pending batch receives requests (#16160)

The consumer will apply the default batch receive policy even if the user will not use the batch receive API.

https://github.com/apache/pulsar/blob/6704f12104219611164aa2bb5bbdfc929613f1bf/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java#L60-L61

This will consume lots of CPU if the client have many consumers (100k consumers)

The Pulsar perf tool can also reproduce the problem if run the test with many consumers

If there is no pending batch receive operation for a consumer, no need to trigger the
batch timeout task periodically. We can only start the timeout check after adding batch
receive request to pending request queue.

Remove the lock in MultiTopicsConsumerImpl as #10352 does

Added new test to verify the batch receive timeout task will not start if no batch
receive request

(cherry picked from commit a0ccdc9)
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Jul 4, 2022
…ving pending batch receives requests (apache#16160)

The consumer will apply the default batch receive policy even if the user will not use the batch receive API.

https://github.com/apache/pulsar/blob/6704f12104219611164aa2bb5bbdfc929613f1bf/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java#L60-L61

This will consume lots of CPU if the client have many consumers (100k consumers)

The Pulsar perf tool can also reproduce the problem if run the test with many consumers

If there is no pending batch receive operation for a consumer, no need to trigger the
batch timeout task periodically. We can only start the timeout check after adding batch
receive request to pending request queue.

Remove the lock in MultiTopicsConsumerImpl as apache#10352 does

Added new test to verify the batch receive timeout task will not start if no batch
receive request

(cherry picked from commit a0ccdc9)
(cherry picked from commit 6ed4ed0)
congbobo184 pushed a commit that referenced this pull request Nov 10, 2022
…ving pending batch receives requests (#16160)

The consumer will apply the default batch receive policy even if the user will not use the batch receive API.

https://github.com/apache/pulsar/blob/6704f12104219611164aa2bb5bbdfc929613f1bf/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java#L60-L61

This will consume lots of CPU if the client have many consumers (100k consumers)

The Pulsar perf tool can also reproduce the problem if run the test with many consumers

If there is no pending batch receive operation for a consumer, no need to trigger the
batch timeout task periodically. We can only start the timeout check after adding batch
receive request to pending request queue.

Remove the lock in MultiTopicsConsumerImpl as #10352 does

Added new test to verify the batch receive timeout task will not start if no batch
receive request

(cherry picked from commit a0ccdc9)
@congbobo184 congbobo184 added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label Nov 10, 2022
Comment on lines -699 to 692
public void redeliverUnacknowledgedMessages() {
lock.writeLock().lock();
try {
internalPinnedExecutor.execute(() -> {
CONSUMER_EPOCH.incrementAndGet(this);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use internalPinnedExecutor then CONSUMER_EPOCH.incrementAndGet(this); the epoch will not work

congbobo184 pushed a commit that referenced this pull request Nov 26, 2022
…ving pending batch receives requests (#16160)

The consumer will apply the default batch receive policy even if the user will not use the batch receive API.

https://github.com/apache/pulsar/blob/6704f12104219611164aa2bb5bbdfc929613f1bf/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java#L60-L61

This will consume lots of CPU if the client have many consumers (100k consumers)

The Pulsar perf tool can also reproduce the problem if run the test with many consumers

If there is no pending batch receive operation for a consumer, no need to trigger the
batch timeout task periodically. We can only start the timeout check after adding batch
receive request to pending request queue.

Remove the lock in MultiTopicsConsumerImpl as #10352 does

Added new test to verify the batch receive timeout task will not start if no batch
receive request

(cherry picked from commit a0ccdc9)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

cherry-picked/branch-2.9 Archived: 2.9 is end of life cherry-picked/branch-2.10 doc-not-needed Your PR changes do not impact docs release/2.9.4 release/2.10.2 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 this pull request may close these issues.

5 participants