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

fix consumer stuck when batchReceivePolicy maxNumMessages > maxReceiverQueueSize #6862

Merged
merged 2 commits into from
May 11, 2020

Conversation

hangc0276
Copy link
Contributor

Fix #6854

Bug description

The consumer stuck due to hasEnoughMessagesForBatchReceive checking:

protected boolean hasEnoughMessagesForBatchReceive() {
        if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) {
            return false;
        }
        return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages())
                || (batchReceivePolicy.getMaxNumBytes() > 0 && INCOMING_MESSAGES_SIZE_UPDATER.get(this) >= batchReceivePolicy.getMaxNumBytes());
    }

Changes

When batchReceivePolicy maxNumMessages > maxReceiverQueueSize, we force batch receivePolicy maxNumMessages to maxReceiverQueueSize to avoid consumer stuck when checking hasEnoughMessagesForBatchReceive

Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

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

I think we should introduce an exception when the maxNumMessages greater than queue size when creating consumers. This will be more transparent to users.

.build();
} else {
this.batchReceivePolicy = conf.getBatchReceivePolicy();
}
} else {
this.batchReceivePolicy = BatchReceivePolicy.DEFAULT_POLICY;
Copy link
Contributor

Choose a reason for hiding this comment

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

We also need to add a check for DEFAULT_POLICY since users may specify the queue size that lower that maxNumMessages of the DEFAULT_POLICY.

Copy link
Contributor

@avimas avimas May 3, 2020

Choose a reason for hiding this comment

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

The above solution is correct but users can get confused seeing their batchRecieve policy configuration has changed
Why not changing the function hasEnoughMessagesForBatchReceive to return true when
incomingMessages.size() >= maxReceiverQueueSize ?

protected boolean hasEnoughMessagesForBatchReceive() {
if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) {
return false;
}
return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages())
|| (batchReceivePolicy.getMaxNumBytes() > 0 && INCOMING_MESSAGES_SIZE_UPDATER.get(this) >= batchReceivePolicy.getMaxNumBytes()) || (incomingMessages.size() >= maxReceiverQueueSize);
}

What do you think ?

Copy link
Contributor

Choose a reason for hiding this comment

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

This will cause batch message return to the user earlier right? This will introduce another problem, users will get the batch message before reaching maxNumMessage. It's better to tell users the receiver queue size needs to greater than the maxNumMessage of the batch receive policy.

Copy link
Contributor

Choose a reason for hiding this comment

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

I completely agree that throwing an exception is a better choice

@codelipenghui codelipenghui added release/2.5.2 type/bug The PR fixed a bug or issue reported a bug labels May 3, 2020
@codelipenghui codelipenghui added this to the 2.6.0 milestone May 3, 2020
@@ -95,7 +95,16 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat
this.schema = schema;
this.interceptors = interceptors;
if (conf.getBatchReceivePolicy() != null) {
this.batchReceivePolicy = conf.getBatchReceivePolicy();
if (conf.getBatchReceivePolicy().getMaxNumMessages() > this.maxReceiverQueueSize) {
Copy link

Choose a reason for hiding this comment

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

can we also add unit tests with the desired behaviour ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for your feedback, I have add the unit tests for all kinds of exception case, please take a look.

@codelipenghui
Copy link
Contributor

@hangc0276 Jia has started a discussion in the email thread for cutting 2.5.2 and it's better to include this fix in 2.5.2. If you have time, please help take a look at the comments, so that we can merge this PR.

@hangc0276
Copy link
Contributor Author

@hangc0276 Jia has started a discussion in the email thread for cutting 2.5.2 and it's better to include this fix in 2.5.2. If you have time, please help take a look at the comments, so that we can merge this PR.

sorry,I will fix this pr soon. When i add test case for the batchReceivePolicy, I found another bugs, and the consumer still stuck when reset the maxNumMessages to receiverQueueSize. I will fix it soon.

@hangc0276
Copy link
Contributor Author

I think we should introduce an exception when the maxNumMessages greater than queue size when creating consumers. This will be more transparent to users.

I have throw a log when reset the batch receive policy, please take a look, thanks.

@hangc0276
Copy link
Contributor Author

@codelipenghui When batch receive triggered by timeout, it should trigger send permit to broker server in MultiTopicsConsumerImpl. Without the trigger, it will also stuck the consumer.

@hangc0276
Copy link
Contributor Author

/pulsarbot run-failure-checks

@jiazhai jiazhai merged commit 561868d into apache:master May 11, 2020
jiazhai pushed a commit that referenced this pull request May 12, 2020
…erQueueSize (#6862)

Fix #6854

### Bug description
The consumer stuck due to `hasEnoughMessagesForBatchReceive` checking:
```
protected boolean hasEnoughMessagesForBatchReceive() {
        if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) {
            return false;
        }
        return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages())
                || (batchReceivePolicy.getMaxNumBytes() > 0 && INCOMING_MESSAGES_SIZE_UPDATER.get(this) >= batchReceivePolicy.getMaxNumBytes());
    }
```

### Changes
When batchReceivePolicy maxNumMessages > maxReceiverQueueSize, we force batch receivePolicy maxNumMessages to maxReceiverQueueSize to avoid consumer stuck when checking `hasEnoughMessagesForBatchReceive`

* fix consumer stuck when batchReceivePolicy maxNumMessages > maxReceiverQueueSize

* throw log warn and add test case
(cherry picked from commit 561868d)
Huanli-Meng pushed a commit to Huanli-Meng/pulsar that referenced this pull request May 27, 2020
…erQueueSize (apache#6862)

Fix apache#6854 

### Bug description
The consumer stuck due to `hasEnoughMessagesForBatchReceive` checking:
```
protected boolean hasEnoughMessagesForBatchReceive() {
        if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) {
            return false;
        }
        return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages())
                || (batchReceivePolicy.getMaxNumBytes() > 0 && INCOMING_MESSAGES_SIZE_UPDATER.get(this) >= batchReceivePolicy.getMaxNumBytes());
    }
```

### Changes
When batchReceivePolicy maxNumMessages > maxReceiverQueueSize, we force batch receivePolicy maxNumMessages to maxReceiverQueueSize to avoid consumer stuck when checking `hasEnoughMessagesForBatchReceive`


* fix consumer stuck when batchReceivePolicy maxNumMessages > maxReceiverQueueSize

* throw log warn and add test case
addisonj pushed a commit to instructure/pulsar that referenced this pull request Jun 12, 2020
…erQueueSize (apache#6862)

Fix apache#6854

### Bug description
The consumer stuck due to `hasEnoughMessagesForBatchReceive` checking:
```
protected boolean hasEnoughMessagesForBatchReceive() {
        if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) {
            return false;
        }
        return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages())
                || (batchReceivePolicy.getMaxNumBytes() > 0 && INCOMING_MESSAGES_SIZE_UPDATER.get(this) >= batchReceivePolicy.getMaxNumBytes());
    }
```

### Changes
When batchReceivePolicy maxNumMessages > maxReceiverQueueSize, we force batch receivePolicy maxNumMessages to maxReceiverQueueSize to avoid consumer stuck when checking `hasEnoughMessagesForBatchReceive`

* fix consumer stuck when batchReceivePolicy maxNumMessages > maxReceiverQueueSize

* throw log warn and add test case
(cherry picked from commit 561868d)
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
…erQueueSize (apache#6862)

Fix apache#6854 

### Bug description
The consumer stuck due to `hasEnoughMessagesForBatchReceive` checking:
```
protected boolean hasEnoughMessagesForBatchReceive() {
        if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) {
            return false;
        }
        return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages())
                || (batchReceivePolicy.getMaxNumBytes() > 0 && INCOMING_MESSAGES_SIZE_UPDATER.get(this) >= batchReceivePolicy.getMaxNumBytes());
    }
```

### Changes
When batchReceivePolicy maxNumMessages > maxReceiverQueueSize, we force batch receivePolicy maxNumMessages to maxReceiverQueueSize to avoid consumer stuck when checking `hasEnoughMessagesForBatchReceive`


* fix consumer stuck when batchReceivePolicy maxNumMessages > maxReceiverQueueSize

* throw log warn and add test case
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
release/2.5.2 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Consumer stuck with batch receive when batchingMaxMessages greater than receiverQueueSize
5 participants