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][client]Fix client memory limit currentUsage leak and semaphore release duplicated in ProducerImpl #16837

Merged
merged 5 commits into from Jul 30, 2022

Conversation

Nicklee007
Copy link
Contributor

@Nicklee007 Nicklee007 commented Jul 28, 2022

Motivation

Fix client memory limit currentUsage leak in ProducerImpl. When our pulsar cluster occur some error, producer send message fail and we find the currentUsage always keep high value like the leaked, and cause the producer send rate is slow.
And find producer semaphore release duplicated when createOpSendMsg occur some excrption.

Follow 1 point only release the message count semaphore, but not release the memory limit.
memory limit currentUsage leak point

final int numMessagesInBatch = batchMessageContainer.getNumMessagesInBatch();
batchMessageContainer.discard(ex);
semaphoreRelease(numMessagesInBatch);

producer semaphore release duplicated

} catch (PulsarClientException e) {
semaphoreRelease(batchMessageContainer.getNumMessagesInBatch());
} catch (Throwable t) {
semaphoreRelease(batchMessageContainer.getNumMessagesInBatch());
log.warn("[{}] [{}] error while create opSendMsg by batch message container", topic, producerName, t);

After the exception the  memory limit leak occured.
org.apache.pulsar.client.api.PulsarClientException$TimeoutException: The producer XXXX-366-15151 can not send message to the topic persistent://XXXX/XXXX/XXXX within given timeout : createdAt 30.005 seconds ago, firstSentAt 30.005 seconds ago, lastSentAt 30.005 seconds ago, retryCount 1
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsg.sendComplete(ProducerImpl.java:1287)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$18(ProducerImpl.java:1826)
        at java.base/java.util.ArrayDeque.forEach(ArrayDeque.java:889)
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsgQueue.forEach(ProducerImpl.java:1369)
        at org.apache.pulsar.client.impl.ProducerImpl.failPendingMessages(ProducerImpl.java:1816)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$19(ProducerImpl.java:1848)
        at org.apache.pulsar.shade.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
        at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
        at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:834)

Modifications

  1. add the MemoryLimitController release.

Documentation

  • doc-not-needed
    (Please explain why)

@codelipenghui
Copy link
Contributor

A related one #16835

@codelipenghui
Copy link
Contributor

@Nicklee007 Could you please help add some unit tests?

@codelipenghui codelipenghui added this to the 2.11.0 milestone Jul 28, 2022
@codelipenghui codelipenghui added type/bug The PR fixed a bug or issue reported a bug release/2.10.2 release/2.9.4 labels Jul 28, 2022
@shoothzj
Copy link
Member

shoothzj commented Jul 28, 2022

@codelipenghui +1 we need some tests to cover this change

@Nicklee007
Copy link
Contributor Author

@Nicklee007 Could you please help add some unit tests?

@codelipenghui @shoothzj Add some unit test, PTAL Thx~

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Jul 29, 2022
@Nicklee007 Nicklee007 requested a review from Jason918 July 29, 2022 02:50
@Nicklee007 Nicklee007 changed the title [fix][client]Fix client memory limit currentUsage leak in ProducerImpl [fix][client]Fix client memory limit currentUsage leak and semaphore release duplicated in ProducerImpl Jul 29, 2022
@Nicklee007
Copy link
Contributor Author

@codelipenghui @Jason918 @shoothzj Also find producer semaphore release duplicated in batchMessageAndSend, if some error cause in batchMessageAndSend, the semaphoreRelease will be invoked in failPendingBatchMessages .
Fixed it and add some unit test to cover it.

Copy link
Contributor

@Jason918 Jason918 left a comment

Choose a reason for hiding this comment

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

LGTM

@shoothzj shoothzj merged commit 955dcd1 into apache:master Jul 30, 2022
Gleiphir2769 pushed a commit to Gleiphir2769/pulsar that referenced this pull request Aug 4, 2022
…release duplicated in ProducerImpl (apache#16837)

### Motivation

Fix client memory limit `currentUsage` leak in `ProducerImpl`. When our pulsar cluster occur some error,  producer send message fail and we find the `currentUsage` always  keep high value like the leaked, and cause the producer send rate is slow.
And find producer semaphore release duplicated when `createOpSendMsg`  occur some excrption.

Follow 1 point only release the message count semaphore, but not release the memory limit.
**memory limit currentUsage leak point**
https://github.com/apache/pulsar/blob/c217b8f559292fd34c6a4fb4b30aab213720d962/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2031-L2033

**producer semaphore release duplicated**
https://github.com/apache/pulsar/blob/4d64e2e66689381ebbb94fbfc03eb4e1dfba0405/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2116-L2120

```
After the exception the  memory limit leak occured.
org.apache.pulsar.client.api.PulsarClientException$TimeoutException: The producer XXXX-366-15151 can not send message to the topic persistent://XXXX/XXXX/XXXX within given timeout : createdAt 30.005 seconds ago, firstSentAt 30.005 seconds ago, lastSentAt 30.005 seconds ago, retryCount 1
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsg.sendComplete(ProducerImpl.java:1287)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$18(ProducerImpl.java:1826)
        at java.base/java.util.ArrayDeque.forEach(ArrayDeque.java:889)
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsgQueue.forEach(ProducerImpl.java:1369)
        at org.apache.pulsar.client.impl.ProducerImpl.failPendingMessages(ProducerImpl.java:1816)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$19(ProducerImpl.java:1848)
        at org.apache.pulsar.shade.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
        at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
        at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:834)

```

### Modifications

1. add the `MemoryLimitController` release.


### Documentation

- [X] `doc-not-needed`
@mattisonchao
Copy link
Member

@Nicklee007 Could you help to open another PR to branch-2.9?

BewareMyPower pushed a commit that referenced this pull request Aug 5, 2022
…release duplicated in ProducerImpl (#16837)

### Motivation

Fix client memory limit `currentUsage` leak in `ProducerImpl`. When our pulsar cluster occur some error,  producer send message fail and we find the `currentUsage` always  keep high value like the leaked, and cause the producer send rate is slow.
And find producer semaphore release duplicated when `createOpSendMsg`  occur some excrption.

Follow 1 point only release the message count semaphore, but not release the memory limit.
**memory limit currentUsage leak point**
https://github.com/apache/pulsar/blob/c217b8f559292fd34c6a4fb4b30aab213720d962/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2031-L2033

**producer semaphore release duplicated**
https://github.com/apache/pulsar/blob/4d64e2e66689381ebbb94fbfc03eb4e1dfba0405/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2116-L2120

```
After the exception the  memory limit leak occured.
org.apache.pulsar.client.api.PulsarClientException$TimeoutException: The producer XXXX-366-15151 can not send message to the topic persistent://XXXX/XXXX/XXXX within given timeout : createdAt 30.005 seconds ago, firstSentAt 30.005 seconds ago, lastSentAt 30.005 seconds ago, retryCount 1
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsg.sendComplete(ProducerImpl.java:1287)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$18(ProducerImpl.java:1826)
        at java.base/java.util.ArrayDeque.forEach(ArrayDeque.java:889)
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsgQueue.forEach(ProducerImpl.java:1369)
        at org.apache.pulsar.client.impl.ProducerImpl.failPendingMessages(ProducerImpl.java:1816)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$19(ProducerImpl.java:1848)
        at org.apache.pulsar.shade.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
        at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
        at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:834)

```

### Modifications

1. add the `MemoryLimitController` release.

### Documentation

- [X] `doc-not-needed`

(cherry picked from commit 955dcd1)
@BewareMyPower
Copy link
Contributor

Remove the release/2.7.5 label because memory limiter was introduced since 2.8, see #8965.

BewareMyPower added a commit that referenced this pull request Aug 5, 2022
…maphore release duplicated in ProducerImpl (#16837)"

Even after importing the mockito dependency, the
testProducerBatchSendTimeoutMemoryRelease will still fail.

This reverts commit 9610640.
@Nicklee007
Copy link
Contributor Author

@Nicklee007 Could you help to open another PR to branch-2.9?

@mattisonchao ok, I'll cherry-pick it to branch 2.9.

@BewareMyPower
Copy link
Contributor

Move this PR to the next release of 2.8.5. For the release manager of next 2.8.x release, this PR requires an extra import of Mockito in #16835. However, even after fixing the compilation error, the test still failed. So let's just delay this PR to the next release.

@Nicklee007 If you want to include this PR in 2.8.4, please open a new PR to migrate this PR to branch-2.8.

@Nicklee007
Copy link
Contributor Author

@Nicklee007 Could you help to open another PR to branch-2.9?

@mattisonchao cherry-pick to PR #16971 in branch 2.9.

@Nicklee007
Copy link
Contributor Author

Remove the release/2.7.5 label because memory limiter was introduced since 2.8, see #8965.

@BewareMyPower @Jason918 Created a new PR #16972 to branch-2.7 which only fix the semaphore release duplicated problem in ProducerImpl, PTAL.

@Nicklee007
Copy link
Contributor Author

Move this PR to the next release of 2.8.5. For the release manager of next 2.8.x release, this PR requires an extra import of Mockito in #16835. However, even after fixing the compilation error, the test still failed. So let's just delay this PR to the next release.

@Nicklee007 If you want to include this PR in 2.8.4, please open a new PR to migrate this PR to branch-2.8.

@BewareMyPower Created a new PR #16985 branch-2.8, PTAL.

codelipenghui pushed a commit that referenced this pull request Aug 8, 2022
…release duplicated in ProducerImpl (#16837)

### Motivation

Fix client memory limit `currentUsage` leak in `ProducerImpl`. When our pulsar cluster occur some error,  producer send message fail and we find the `currentUsage` always  keep high value like the leaked, and cause the producer send rate is slow.
And find producer semaphore release duplicated when `createOpSendMsg`  occur some excrption.

Follow 1 point only release the message count semaphore, but not release the memory limit.
**memory limit currentUsage leak point**
https://github.com/apache/pulsar/blob/c217b8f559292fd34c6a4fb4b30aab213720d962/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2031-L2033

**producer semaphore release duplicated**
https://github.com/apache/pulsar/blob/4d64e2e66689381ebbb94fbfc03eb4e1dfba0405/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2116-L2120

```
After the exception the  memory limit leak occured.
org.apache.pulsar.client.api.PulsarClientException$TimeoutException: The producer XXXX-366-15151 can not send message to the topic persistent://XXXX/XXXX/XXXX within given timeout : createdAt 30.005 seconds ago, firstSentAt 30.005 seconds ago, lastSentAt 30.005 seconds ago, retryCount 1
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsg.sendComplete(ProducerImpl.java:1287)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$18(ProducerImpl.java:1826)
        at java.base/java.util.ArrayDeque.forEach(ArrayDeque.java:889)
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsgQueue.forEach(ProducerImpl.java:1369)
        at org.apache.pulsar.client.impl.ProducerImpl.failPendingMessages(ProducerImpl.java:1816)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$19(ProducerImpl.java:1848)
        at org.apache.pulsar.shade.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
        at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
        at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:834)

```

### Modifications

1. add the `MemoryLimitController` release.

### Documentation

- [X] `doc-not-needed`

(cherry picked from commit 955dcd1)
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Aug 16, 2022
…release duplicated in ProducerImpl (apache#16837)

### Motivation

Fix client memory limit `currentUsage` leak in `ProducerImpl`. When our pulsar cluster occur some error,  producer send message fail and we find the `currentUsage` always  keep high value like the leaked, and cause the producer send rate is slow.
And find producer semaphore release duplicated when `createOpSendMsg`  occur some excrption.

Follow 1 point only release the message count semaphore, but not release the memory limit.
**memory limit currentUsage leak point**
https://github.com/apache/pulsar/blob/c217b8f559292fd34c6a4fb4b30aab213720d962/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2031-L2033

**producer semaphore release duplicated**
https://github.com/apache/pulsar/blob/4d64e2e66689381ebbb94fbfc03eb4e1dfba0405/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2116-L2120

```
After the exception the  memory limit leak occured.
org.apache.pulsar.client.api.PulsarClientException$TimeoutException: The producer XXXX-366-15151 can not send message to the topic persistent://XXXX/XXXX/XXXX within given timeout : createdAt 30.005 seconds ago, firstSentAt 30.005 seconds ago, lastSentAt 30.005 seconds ago, retryCount 1
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsg.sendComplete(ProducerImpl.java:1287)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$18(ProducerImpl.java:1826)
        at java.base/java.util.ArrayDeque.forEach(ArrayDeque.java:889)
        at org.apache.pulsar.client.impl.ProducerImpl$OpSendMsgQueue.forEach(ProducerImpl.java:1369)
        at org.apache.pulsar.client.impl.ProducerImpl.failPendingMessages(ProducerImpl.java:1816)
        at org.apache.pulsar.client.impl.ProducerImpl.lambda$failPendingMessages$19(ProducerImpl.java:1848)
        at org.apache.pulsar.shade.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
        at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
        at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
        at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:834)

```

### Modifications

1. add the `MemoryLimitController` release.

### Documentation

- [X] `doc-not-needed`

(cherry picked from commit 955dcd1)
(cherry picked from commit 7c73269)
@zymap zymap added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Sep 15, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/client cherry-picked/branch-2.8 Archived: 2.8 is end of life cherry-picked/branch-2.10 doc-not-needed Your PR changes do not impact docs release/2.8.5 release/2.10.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.

None yet