You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by ji...@apache.org on 2022/09/26 03:34:28 UTC

[pulsar] branch branch-2.10 updated: [fix][broker]Consumer can't consume messages because there has two sames topics in one broker (#17526)

This is an automated email from the ASF dual-hosted git repository.

jianghaiting pushed a commit to branch branch-2.10
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.10 by this push:
     new ddd642e77de [fix][broker]Consumer can't consume messages because there has two sames topics in one broker (#17526)
ddd642e77de is described below

commit ddd642e77deacc42ad2542df39cbb22ef5217b25
Author: fengyubiao <yu...@streamnative.io>
AuthorDate: Wed Sep 21 21:28:38 2022 +0800

    [fix][broker]Consumer can't consume messages because there has two sames topics in one broker (#17526)
    
    (cherry picked from commit 260f5c65e9937ede345d84777debdc8f7b571e1f)
---
 .../pulsar/broker/service/BrokerService.java       | 56 ++++++++++++++++++++--
 .../service/nonpersistent/NonPersistentTopic.java  |  4 +-
 .../broker/service/persistent/PersistentTopic.java |  4 +-
 3 files changed, 55 insertions(+), 9 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 43ce75ec31a..fb2bc3bbe1a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -1913,7 +1913,7 @@ public class BrokerService implements Closeable {
             TopicName topicName = TopicName.get(topic);
             if (serviceUnit.includes(topicName) && getTopicReference(topic).isPresent()) {
                 log.info("[{}][{}] Clean unloaded topic from cache.", serviceUnit.toString(), topic);
-                pulsar.getBrokerService().removeTopicFromCache(topicName.toString(), serviceUnit);
+                pulsar.getBrokerService().removeTopicFromCache(topicName.toString(), serviceUnit, null);
             }
         }
     }
@@ -1922,15 +1922,56 @@ public class BrokerService implements Closeable {
         return authorizationService;
     }
 
-    public CompletableFuture<Void> removeTopicFromCache(String topic) {
+    public CompletableFuture<Void> removeTopicFromCache(String topicName) {
+        return removeTopicFutureFromCache(topicName, null);
+    }
+
+    public CompletableFuture<Void> removeTopicFromCache(Topic topic) {
+        Optional<CompletableFuture<Optional<Topic>>> createTopicFuture = findTopicFutureInCache(topic);
+        if (!createTopicFuture.isPresent()){
+            return CompletableFuture.completedFuture(null);
+        }
+        return removeTopicFutureFromCache(topic.getName(), createTopicFuture.get());
+    }
+
+    private Optional<CompletableFuture<Optional<Topic>>> findTopicFutureInCache(Topic topic){
+        if (topic == null){
+            return Optional.empty();
+        }
+        final CompletableFuture<Optional<Topic>> createTopicFuture = topics.get(topic.getName());
+        // If not exists in cache, do nothing.
+        if (createTopicFuture == null){
+            return Optional.empty();
+        }
+        // If the future in cache is not yet complete, the topic instance in the cache is not the same with the topic.
+        if (!createTopicFuture.isDone()){
+            return Optional.empty();
+        }
+        // If the future in cache has exception complete,
+        // the topic instance in the cache is not the same with the topic.
+        if (createTopicFuture.isCompletedExceptionally()){
+            return Optional.empty();
+        }
+        Optional<Topic> optionalTopic = createTopicFuture.join();
+        Topic topicInCache = optionalTopic.orElse(null);
+        if (topicInCache == null || topicInCache != topic){
+            return Optional.empty();
+        } else {
+            return Optional.of(createTopicFuture);
+        }
+    }
+
+    private CompletableFuture<Void> removeTopicFutureFromCache(String topic,
+                                                        CompletableFuture<Optional<Topic>> createTopicFuture) {
         TopicName topicName = TopicName.get(topic);
         return pulsar.getNamespaceService().getBundleAsync(topicName)
                 .thenAccept(namespaceBundle -> {
-                    removeTopicFromCache(topic, namespaceBundle);
+                    removeTopicFromCache(topic, namespaceBundle, createTopicFuture);
                 });
     }
 
-    public void removeTopicFromCache(String topic, NamespaceBundle namespaceBundle) {
+    private void removeTopicFromCache(String topic, NamespaceBundle namespaceBundle,
+                                     CompletableFuture<Optional<Topic>> createTopicFuture) {
         String bundleName = namespaceBundle.toString();
         String namespaceName = TopicName.get(topic).getNamespaceObject().toString();
 
@@ -1957,7 +1998,12 @@ public class BrokerService implements Closeable {
                 }
             }
         }
-        topics.remove(topic);
+
+        if (createTopicFuture == null) {
+            topics.remove(topic);
+        } else {
+            topics.remove(topic, createTopicFuture);
+        }
 
         Compactor compactor = pulsar.getNullableCompactor();
         if (compactor != null) {
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 bb62cf0f39f..bd553ec5ca9 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
@@ -440,7 +440,7 @@ public class NonPersistentTopic extends AbstractTopic implements Topic, TopicPol
                             // topic GC iterates over topics map and removing from the map with the same thread creates
                             // deadlock. so, execute it in different thread
                             brokerService.executor().execute(() -> {
-                                brokerService.removeTopicFromCache(topic);
+                                brokerService.removeTopicFromCache(NonPersistentTopic.this);
                                 unregisterTopicPolicyListener();
                                 log.info("[{}] Topic deleted", topic);
                                 deleteFuture.complete(null);
@@ -507,7 +507,7 @@ public class NonPersistentTopic extends AbstractTopic implements Topic, TopicPol
             // unload topic iterates over topics map and removing from the map with the same thread creates deadlock.
             // so, execute it in different thread
             brokerService.executor().execute(() -> {
-                brokerService.removeTopicFromCache(topic);
+                brokerService.removeTopicFromCache(NonPersistentTopic.this);
                 unregisterTopicPolicyListener();
                 closeFuture.complete(null);
             });
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
index 835e198ef76..44c7c962d1e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -1185,7 +1185,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
                                     ledger.asyncDelete(new AsyncCallbacks.DeleteLedgerCallback() {
                                         @Override
                                         public void deleteLedgerComplete(Object ctx) {
-                                            brokerService.removeTopicFromCache(topic);
+                                            brokerService.removeTopicFromCache(PersistentTopic.this);
 
                                             dispatchRateLimiter.ifPresent(DispatchRateLimiter::close);
 
@@ -1305,7 +1305,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
     }
 
     private void disposeTopic(CompletableFuture<?> closeFuture) {
-        brokerService.removeTopicFromCache(topic)
+        brokerService.removeTopicFromCache(PersistentTopic.this)
                 .thenRun(() -> {
                     replicatedSubscriptionsController.ifPresent(ReplicatedSubscriptionsController::close);