This is an automated email from the ASF dual-hosted git repository. penghui pushed a commit to branch branch-2.7 in repository https://gitbox.apache.org/repos/asf/pulsar.git
commit e44a30866056aeaf991044973153da25d54ad5d2 Author: Michal Koziorowski <[email protected]> AuthorDate: Mon Dec 7 03:37:20 2020 +0100 Execute removing non-persistent subscription of topic from different thread to avoid deadlock when removing inactive subscriptions. (#8820) Fixes #8783 Execute removing non-persistent subscription of topic from different thread to avoid deadlock when removing inactive subscriptions. ### Motivation We have non persistent topics with configured removing inactive subscriptions. We've noticed, that removing subscriptions sometimes hangs. After investigation, we've noticed that it hangs on remove operation of ConcurrentOpenHashMap. We've found that removal operation is made inside lambda of forEach method on same collection. Problem was causing not only problems with subscription removal, but after it happened once, problems with creating new subscriptions. ### Modifications We've modified code and run remove operation in other thread to avoid deadlock. (cherry picked from commit 673b4f66bcfb5b041aa78d4c69dd5b256fddf2b1) --- .../pulsar/broker/service/nonpersistent/NonPersistentTopic.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index b2a065c..340df37 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -980,8 +980,9 @@ public class NonPersistentTopic extends AbstractTopic implements Topic { @Override public CompletableFuture<Void> unsubscribe(String subscriptionName) { - subscriptions.remove(subscriptionName); - return CompletableFuture.completedFuture(null); + // checkInactiveSubscriptions iterates over subscriptions map and removing from the map with the same thread. + // That creates deadlock. so, execute remove it in different thread. + return CompletableFuture.runAsync(() -> subscriptions.remove(subscriptionName), brokerService.executor()); } @Override
