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

[pulsar-broker] Close previous dispatcher when subscription type changes #5288

Merged
merged 2 commits into from
Oct 1, 2019

Conversation

massakam
Copy link
Contributor

Motivation

If the subscription type on a topic changes, a new dispatcher is created and the old one is discarded. However, this old dispatcher is not closed. This will cause a memory leak.

Modifications

When the type of a subscription changes and a new dispatcher is created, close the previous one.

@massakam massakam added type/bug The PR fixed a bug or issue reported a bug area/broker labels Sep 28, 2019
@massakam massakam added this to the 2.4.2 milestone Sep 28, 2019
@massakam massakam self-assigned this Sep 28, 2019
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 noticed if all consumer disconnected in a subscription, the topic will remove this subscription and the dispatcher is maintained by the subscription. So why don't close previous dispatcher will cause a memory leak, maybe i have not get your point here.

Copy link
Contributor

@merlimat merlimat left a comment

Choose a reason for hiding this comment

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

👍

@@ -64,6 +64,10 @@ public synchronized boolean canUnsubscribe(Consumer consumer) {
return consumerList.size() == 1 && consumerSet.contains(consumer);
}

public boolean isClosed() {
return IS_CLOSED_UPDATER.get(this) == TRUE;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: for regular get/set operations, we can just use the variable itself return isClosed == TRUE

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

@massakam
Copy link
Contributor Author

@codelipenghui

I noticed if all consumer disconnected in a subscription, the topic will remove this subscription and the dispatcher is maintained by the subscription. So why don't close previous dispatcher will cause a memory leak, maybe i have not get your point here.

The problem is that the rate limiter used for dispatch throttling is only closed in the close() method of dispatcher.

if (dispatchRateLimiter.isPresent()) {
dispatchRateLimiter.get().close();
}

public void close() {
// close rate-limiter
if (dispatchRateLimiterOnMessage != null) {
dispatchRateLimiterOnMessage.close();
dispatchRateLimiterOnMessage = null;
}
if (dispatchRateLimiterOnByte != null) {
dispatchRateLimiterOnByte.close();
dispatchRateLimiterOnByte = null;
}
}

And if the RateLimiter object is not closed, the renewTask executed every second will continue to run without being cancelled. Therefore, even if the dispatcher is garbage collected, RateLimiter is not removed.

@Override
public synchronized void close() {
if (!isClosed) {
if (!externalExecutor) {
executorService.shutdownNow();
}
if (renewTask != null) {
renewTask.cancel(false);
}
isClosed = true;
}
}

@codelipenghui
Copy link
Contributor

@massakam Got it, thanks. why don't we close the dispatcher at removeConsumer

@massakam
Copy link
Contributor Author

@codelipenghui Hmm, if the cursor is not durable, the subscription is closed, so the dispatcher should also be closed and I think it should be fixed.

if (!cursor.isDurable()) {
// If cursor is not durable, we need to clean up the subscription as well
close();

I will fix it in another pull-request.

@merlimat
Copy link
Contributor

@massakam Got it, thanks. why don't we close the dispatcher at removeConsumer

when consumers are removed, it might be very likely that they're going to get reconnected immediately thereafter. if the subscription type is not changed, dispatcher will still be valid

@codelipenghui
Copy link
Contributor

retest this please

@massakam
Copy link
Contributor Author

massakam commented Oct 1, 2019

rerun cpp tests
rerun integration tests

@aahmed-se aahmed-se merged commit 413ba03 into apache:master Oct 1, 2019
@massakam massakam deleted the close-dispatcher branch October 1, 2019 07:52
jiazhai pushed a commit that referenced this pull request Oct 9, 2019
…oved from subscription (#5340)

### Motivation

If cursor is not durable, the subscription is closed and removed from the topic when all consumers are removed. 
https://github.com/apache/pulsar/blob/413ba03050036a620fa346456ef6c3ff6071e9ab/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java#L237-L254

As mentioned in #5288, the dispatcher also needs to be closed at this time. Otherwise, `RateLimiter` instances will not be garbage collected, causing a memory leak.
wolfstudy pushed a commit that referenced this pull request Nov 20, 2019
…ges (#5288)

* Close previous dispatcher when subscription type changes

* Use isClosed directly to check if dispatcher is closed

(cherry picked from commit 413ba03)
wolfstudy pushed a commit that referenced this pull request Nov 20, 2019
…oved from subscription (#5340)

### Motivation

If cursor is not durable, the subscription is closed and removed from the topic when all consumers are removed. 
https://github.com/apache/pulsar/blob/413ba03050036a620fa346456ef6c3ff6071e9ab/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java#L237-L254

As mentioned in #5288, the dispatcher also needs to be closed at this time. Otherwise, `RateLimiter` instances will not be garbage collected, causing a memory leak.

(cherry picked from commit 136048c)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker 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.

5 participants