[fix][broker]Consumer can't consume messages because there has two sames topics in one broker (#17526)
(cherry picked from commit 260f5c65e9937ede345d84777debdc8f7b571e1f)
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 43ce75e..fb2bc3b 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 @@
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 @@
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 @@
}
}
}
- 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 bb62cf0..bd553ec 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 @@
// 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 @@
// 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 835e198e..44c7c96 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 @@
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 @@
}
private void disposeTopic(CompletableFuture<?> closeFuture) {
- brokerService.removeTopicFromCache(topic)
+ brokerService.removeTopicFromCache(PersistentTopic.this)
.thenRun(() -> {
replicatedSubscriptionsController.ifPresent(ReplicatedSubscriptionsController::close);