Skip to content

Conversation

@avimas
Copy link
Contributor

@avimas avimas commented May 3, 2020

Batch size is not limited to the minimum of the maxNumberOfMessages and maxSizeOfMessages from the BatchRecieve policy.

  1. Batch size is not limited to the minimum of the maxNumberOfMessages and maxSizeOfMessages from the BatchRecieve policy.
    I think this issue can be easly fixed by changing the canAdd function in MessagesImpl from:
protected boolean canAdd(Message<T> message) {
    if (this.maxNumberOfMessages <= 0 && this.maxSizeOfMessages <= 0L) {
        return true;
    } else {
        return this.maxNumberOfMessages > 0 && this.currentNumberOfMessages + 1 <= this.maxNumberOfMessages || this.maxSizeOfMessages > 0L &&   this.currentSizeOfMessages + (long)message.getData().length <= this.maxSizeOfMessages;
    }
}

to (changing the condintion in the else to && instead of ||):

protected boolean canAdd(Message<T> message) {
    if (this.maxNumberOfMessages <= 0 && this.maxSizeOfMessages <= 0L) {
        return true;
    } else {
        return (this.maxNumberOfMessages > 0 && this.currentNumberOfMessages + 1 <= this.maxNumberOfMessages) && (this.maxSizeOfMessages > 0L &&   this.currentSizeOfMessages + (long)message.getData().length <= this.maxSizeOfMessages);
    }
}
  1. When the batch size is higher than the recieveQ of the consumer (I used a batch size of 3000 and a receiveQ of 500) I noticed the following issues:
    a. In a mutliTopic (pattern) consumer the client stops receiving any messages I think it getting paused and never resumed when setting a timeout in the batch policy, only one batch is fetched and the client never resumed.

talked with @codelipenghui and advised to open this pull request

@codelipenghui codelipenghui requested review from jiazhai and sijie May 3, 2020 12:57
@codelipenghui codelipenghui added the type/bug The PR fixed a bug or issue reported a bug label May 3, 2020
@codelipenghui codelipenghui added this to the 2.6.0 milestone May 3, 2020
@codelipenghui
Copy link
Contributor

/pulsarbot run-failure-checks

9 similar comments
@codelipenghui
Copy link
Contributor

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 11, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 11, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 11, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 12, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 12, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 12, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 12, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 12, 2020

/pulsarbot run-failure-checks

@jiazhai jiazhai closed this May 12, 2020
@jiazhai jiazhai reopened this May 12, 2020
@jiazhai
Copy link
Member

jiazhai commented May 13, 2020

/pulsarbot run-failure-checks

2 similar comments
@jiazhai
Copy link
Member

jiazhai commented May 13, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 13, 2020

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented May 13, 2020

would like to remove it from 2.5.2

@jiazhai
Copy link
Member

jiazhai commented May 13, 2020

/pulsarbot run-failure-checks

2 similar comments
@jiazhai
Copy link
Member

jiazhai commented May 13, 2020

/pulsarbot run-failure-checks

@codelipenghui
Copy link
Contributor

/pulsarbot run-failure-checks

@codelipenghui codelipenghui changed the title limiting batch size to the minimum of the maxNumberOfMessages and max… limiting batch size to the minimum of the maxNumberOfMessages and maxSizeOfMessages Jun 4, 2020
@codelipenghui
Copy link
Contributor

@avimas Could you please help rebase to the master branch? There are some CI test fix on master branch.

@avimas
Copy link
Contributor Author

avimas commented Jun 4, 2020

@avimas Could you please help rebase to the master branch? There are some CI test fix on master branch.

Done!

@codelipenghui
Copy link
Contributor

@avimas Please rebase again, I can't push to your master branch. It's better to checkout -b a new branch to create a pull request. #7173 has fix some issues with CI tests.

@avimas
Copy link
Contributor Author

avimas commented Jun 5, 2020

@codelipenghui rebase done, I'll make sure to use a new branch next time.

@codelipenghui
Copy link
Contributor

/pulsarbot run-failure-checks

@codelipenghui
Copy link
Contributor

@avimas Could you please take a look at the failed CI tests? look related to this PR.

@codelipenghui
Copy link
Contributor

move to 2.7.0 first.

@sijie
Copy link
Member

sijie commented Jun 10, 2020

/pulsarbot run-failure-checks

1 similar comment
@codelipenghui
Copy link
Contributor

/pulsarbot run-failure-checks

@codelipenghui codelipenghui merged commit 941ddd6 into apache:master Jun 10, 2020
wolfstudy pushed a commit that referenced this pull request Jul 29, 2020
…SizeOfMessages (#6865)

Batch size is not limited to the minimum of the maxNumberOfMessages and maxSizeOfMessages from the BatchRecieve policy.

 1. Batch size is not limited to the minimum of the maxNumberOfMessages and maxSizeOfMessages from the BatchRecieve policy.
I think this issue can be easly fixed by changing the canAdd function in MessagesImpl from:
```
protected boolean canAdd(Message<T> message) {
    if (this.maxNumberOfMessages <= 0 && this.maxSizeOfMessages <= 0L) {
        return true;
    } else {
        return this.maxNumberOfMessages > 0 && this.currentNumberOfMessages + 1 <= this.maxNumberOfMessages || this.maxSizeOfMessages > 0L &&   this.currentSizeOfMessages + (long)message.getData().length <= this.maxSizeOfMessages;
    }
}
```
to (changing the condintion in the else to && instead of ||):
```
protected boolean canAdd(Message<T> message) {
    if (this.maxNumberOfMessages <= 0 && this.maxSizeOfMessages <= 0L) {
        return true;
    } else {
        return (this.maxNumberOfMessages > 0 && this.currentNumberOfMessages + 1 <= this.maxNumberOfMessages) && (this.maxSizeOfMessages > 0L &&   this.currentSizeOfMessages + (long)message.getData().length <= this.maxSizeOfMessages);
    }
}
```

2. When the batch size is higher than the recieveQ of the consumer (I used a batch size of 3000 and a receiveQ of 500) I noticed the following issues:
	a. In a mutliTopic (pattern) consumer the client stops receiving any messages I think it getting paused and never resumed when setting a timeout in the batch policy, only one batch is fetched and the client never resumed.

talked with @codelipenghui  and advised to open this pull request

(cherry picked from commit 941ddd6)
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
…SizeOfMessages (apache#6865)

Batch size is not limited to the minimum of the maxNumberOfMessages and maxSizeOfMessages from the BatchRecieve policy.

 1. Batch size is not limited to the minimum of the maxNumberOfMessages and maxSizeOfMessages from the BatchRecieve policy.
I think this issue can be easly fixed by changing the canAdd function in MessagesImpl from:
```
protected boolean canAdd(Message<T> message) {
    if (this.maxNumberOfMessages <= 0 && this.maxSizeOfMessages <= 0L) {
        return true;
    } else {
        return this.maxNumberOfMessages > 0 && this.currentNumberOfMessages + 1 <= this.maxNumberOfMessages || this.maxSizeOfMessages > 0L &&   this.currentSizeOfMessages + (long)message.getData().length <= this.maxSizeOfMessages;
    }
}
```
to (changing the condintion in the else to && instead of ||):
```
protected boolean canAdd(Message<T> message) {
    if (this.maxNumberOfMessages <= 0 && this.maxSizeOfMessages <= 0L) {
        return true;
    } else {
        return (this.maxNumberOfMessages > 0 && this.currentNumberOfMessages + 1 <= this.maxNumberOfMessages) && (this.maxSizeOfMessages > 0L &&   this.currentSizeOfMessages + (long)message.getData().length <= this.maxSizeOfMessages);
    }
}
```

2. When the batch size is higher than the recieveQ of the consumer (I used a batch size of 3000 and a receiveQ of 500) I noticed the following issues:
	a. In a mutliTopic (pattern) consumer the client stops receiving any messages I think it getting paused and never resumed when setting a timeout in the batch policy, only one batch is fetched and the client never resumed.

talked with @codelipenghui  and advised to open this pull request
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

release/2.6.1 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.

4 participants