Skip to content

Commit

Permalink
If cursor is not durable, close dispatcher when all consumers are rem…
Browse files Browse the repository at this point in the history
…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)
  • Loading branch information
Masahiro Sakamoto authored and wolfstudy committed Nov 19, 2019
1 parent edbb353 commit 4a6f1d3
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,22 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE

if (!cursor.isDurable()) {
// If cursor is not durable, we need to clean up the subscription as well
close();
this.close().thenRun(() -> {
synchronized (this) {
if (dispatcher != null) {
dispatcher.close().thenRun(() -> {
log.info("[{}][{}] Successfully closed dispatcher for reader", topicName, subName);
}).exceptionally(ex -> {
log.error("[{}][{}] Failed to close dispatcher for reader", topicName, subName, ex);
return null;
});
}
}
}).exceptionally(exception -> {
log.error("[{}][{}] Failed to close subscription for reader", topicName, subName, exception);
return null;
});

// when topic closes: it iterates through concurrent-subscription map to close each subscription. so,
// topic.remove again try to access same map which creates deadlock. so, execute it in different thread.
topic.getBrokerService().pulsar().getExecutor().submit(() ->{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -539,6 +539,30 @@ public void testAddRemoveConsumer() throws Exception {
}
}

@Test
public void testAddRemoveConsumerDurableCursor() throws Exception {
doReturn(false).when(cursorMock).isDurable();

PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService);
PersistentSubscription sub = new PersistentSubscription(topic, "non-durable-sub", cursorMock, false);

Consumer consumer = new Consumer(sub, SubType.Exclusive, topic.getName(), 1, 0, "Cons1", 50000, serverCnx,
"myrole-1", Collections.emptyMap(), false, InitialPosition.Latest);

sub.addConsumer(consumer);
assertFalse(sub.getDispatcher().isClosed());
sub.removeConsumer(consumer);

// The dispatcher is closed asynchronously
for (int i = 0; i < 100; i++) {
if (sub.getDispatcher().isClosed()) {
break;
}
Thread.sleep(100);
}
assertTrue(sub.getDispatcher().isClosed());
}

public void testMaxConsumersShared() throws Exception {
PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService);
PersistentSubscription sub = new PersistentSubscription(topic, "sub-1", cursorMock, false);
Expand Down

0 comments on commit 4a6f1d3

Please sign in to comment.