diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 2e9f9dc6b0105..6cbc99e2cf4d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1964,6 +1964,11 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception return new BrokerService(pulsar, ioEventLoopGroup); } + @VisibleForTesting + public void setTransactionExecutorProvider(TransactionBufferProvider transactionBufferProvider) { + this.transactionBufferProvider = transactionBufferProvider; + } + private CompactionServiceFactory loadCompactionServiceFactory() { String compactionServiceFactoryClassName = config.getCompactionServiceFactoryClassName(); var compactionServiceFactory = 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 9a08578ee4088..82d7fad38740e 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 @@ -1001,38 +1001,38 @@ public CompletableFuture> getTopic(final String topic, boolean c return getTopic(TopicName.get(topic), createIfMissing, properties); } + /** + * Retrieves or creates a topic based on the specified parameters. + * 0. If disable PersistentTopics or NonPersistentTopics, it will return a failed future with NotAllowedException. + * 1. If topic future exists in the cache returned directly regardless of whether it fails or timeout. + * 2. If the topic metadata exists, the topic is created regardless of {@code createIfMissing}. + * 3. If the topic metadata not exists, and {@code createIfMissing} is false, + * returns an empty Optional in a CompletableFuture. And this empty future not be added to the map. + * 4. Otherwise, use computeIfAbsent. It returns the existing topic or creates and adds a new topicFuture. + * Any exceptions will remove the topicFuture from the map. + * + * @param topicName The name of the topic, potentially including partition information. + * @param createIfMissing If true, creates the topic if it does not exist. + * @param properties Topic configuration properties used during creation. + * @return CompletableFuture with an Optional of the topic if found or created, otherwise empty. + */ public CompletableFuture> getTopic(final TopicName topicName, boolean createIfMissing, Map properties) { try { - CompletableFuture> topicFuture = topics.get(topicName.toString()); - if (topicFuture != null) { - if (topicFuture.isCompletedExceptionally() - || (topicFuture.isDone() && !topicFuture.getNow(Optional.empty()).isPresent())) { - // Exceptional topics should be recreated. - topics.remove(topicName.toString(), topicFuture); - } else { - // a non-existing topic in the cache shouldn't prevent creating a topic - if (createIfMissing) { - if (topicFuture.isDone() && topicFuture.getNow(Optional.empty()).isPresent()) { - return topicFuture; - } else { - return topicFuture.thenCompose(value -> { - if (!value.isPresent()) { - // retry and create topic - return getTopic(topicName, createIfMissing, properties); - } else { - // in-progress future completed successfully - return CompletableFuture.completedFuture(value); - } - }); - } - } else { - return topicFuture; - } - } + // If topic future exists in the cache returned directly regardless of whether it fails or timeout. + CompletableFuture> tp = topics.get(topicName.toString()); + if (tp != null) { + return tp; } final boolean isPersistentTopic = topicName.getDomain().equals(TopicDomain.persistent); if (isPersistentTopic) { + if (!pulsar.getConfiguration().isEnablePersistentTopics()) { + if (log.isDebugEnabled()) { + log.debug("Broker is unable to load persistent topic {}", topicName); + } + return FutureUtil.failedFuture(new NotAllowedException( + "Broker is unable to load persistent topic")); + } return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topicName) .thenCompose(exists -> { if (!exists && !createIfMissing) { @@ -1047,44 +1047,48 @@ public CompletableFuture> getTopic(final TopicName topicName, bo throw FutureUtil.wrapToCompletionException(new ServiceUnitNotReadyException(errorInfo)); }).thenCompose(optionalTopicPolicies -> { final TopicPolicies topicPolicies = optionalTopicPolicies.orElse(null); - return topics.computeIfAbsent(topicName.toString(), (tpName) -> { - if (topicName.isPartitioned()) { - final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); - return fetchPartitionedTopicMetadataAsync(topicNameEntity) - .thenCompose((metadata) -> { - // Allow crate non-partitioned persistent topic that name includes - // `partition` - if (metadata.partitions == 0 - || topicName.getPartitionIndex() < metadata.partitions) { - return loadOrCreatePersistentTopic(tpName, createIfMissing, - properties, topicPolicies); - } + if (topicName.isPartitioned()) { + final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); + return fetchPartitionedTopicMetadataAsync(topicNameEntity) + .thenCompose((metadata) -> { + // Allow crate non-partitioned persistent topic that name includes + // `partition` + if (metadata.partitions == 0 + || topicName.getPartitionIndex() < metadata.partitions) { + return topics.computeIfAbsent(topicName.toString(), (tpName) -> + loadOrCreatePersistentTopic(tpName, + createIfMissing, properties, topicPolicies)); + } else { final String errorMsg = String.format("Illegal topic partition name %s with max allowed " + "%d partitions", topicName, metadata.partitions); log.warn(errorMsg); return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException(errorMsg)); - }); - } - return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); - }).thenCompose(optionalTopic -> { - if (!optionalTopic.isPresent() && createIfMissing) { - log.warn("[{}] Try to recreate the topic with createIfMissing=true " - + "but the returned topic is empty", topicName); - return getTopic(topicName, createIfMissing, properties); - } - return CompletableFuture.completedFuture(optionalTopic); - }); + } + }); + } else { + return topics.computeIfAbsent(topicName.toString(), (tpName) -> + loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies)); + } }); }); } else { - return topics.computeIfAbsent(topicName.toString(), (name) -> { + if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { + if (log.isDebugEnabled()) { + log.debug("Broker is unable to load non-persistent topic {}", topicName); + } + return FutureUtil.failedFuture(new NotAllowedException( + "Broker is unable to load persistent topic")); + } + if (!topics.containsKey(topicName.toString())) { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.BEFORE); - if (topicName.isPartitioned()) { - final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); - return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> { - if (topicName.getPartitionIndex() < metadata.partitions) { + } + if (topicName.isPartitioned()) { + final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); + return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> { + if (topicName.getPartitionIndex() < metadata.partitions) { + return topics.computeIfAbsent(topicName.toString(), (name) -> { topicEventsDispatcher .notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE); @@ -1095,11 +1099,13 @@ public CompletableFuture> getTopic(final TopicName topicName, bo topicEventsDispatcher .notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD); return res; - } - topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); - return CompletableFuture.completedFuture(Optional.empty()); - }); - } else if (createIfMissing) { + }); + } + topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); + return CompletableFuture.completedFuture(Optional.empty()); + }); + } else if (createIfMissing) { + return topics.computeIfAbsent(topicName.toString(), (name) -> { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE); CompletableFuture> res = createNonPersistentTopic(name); @@ -1109,11 +1115,15 @@ public CompletableFuture> getTopic(final TopicName topicName, bo topicEventsDispatcher .notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD); return res; - } else { + }); + } else { + CompletableFuture> topicFuture = topics.get(topicName.toString()); + if (topicFuture == null) { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); - return CompletableFuture.completedFuture(Optional.empty()); + topicFuture = CompletableFuture.completedFuture(Optional.empty()); } - }); + return topicFuture; + } } } catch (IllegalArgumentException e) { log.warn("[{}] Illegalargument exception when loading topic", topicName, e); @@ -1252,15 +1262,9 @@ private CompletableFuture> createNonPersistentTopic(String topic CompletableFuture> topicFuture = new CompletableFuture<>(); topicFuture.exceptionally(t -> { pulsarStats.recordTopicLoadFailed(); + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); return null; }); - if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { - if (log.isDebugEnabled()) { - log.debug("Broker is unable to load non-persistent topic {}", topic); - } - return FutureUtil.failedFuture( - new NotAllowedException("Broker is not unable to load non-persistent topic")); - } final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); NonPersistentTopic nonPersistentTopic; try { @@ -1283,7 +1287,6 @@ private CompletableFuture> createNonPersistentTopic(String topic }).exceptionally(ex -> { log.warn("Replication check failed. Removing topic from topics list {}, {}", topic, ex.getCause()); nonPersistentTopic.stopReplProducers().whenComplete((v, exception) -> { - pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex); }); return null; @@ -1534,14 +1537,6 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S final CompletableFuture> topicFuture = FutureUtil.createFutureWithTimeout( Duration.ofSeconds(pulsar.getConfiguration().getTopicLoadTimeoutSeconds()), executor(), () -> FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION); - if (!pulsar.getConfiguration().isEnablePersistentTopics()) { - if (log.isDebugEnabled()) { - log.debug("Broker is unable to load persistent topic {}", topic); - } - topicFuture.completeExceptionally(new NotAllowedException( - "Broker is unable to load persistent topic")); - return topicFuture; - } checkTopicNsOwnership(topic) .thenRun(() -> { @@ -1556,6 +1551,7 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S // do not recreate topic if topic is already migrated and deleted by broker // so, avoid creating a new topic if migration is already started if (ex != null && (ex.getCause() instanceof TopicMigratedException)) { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex.getCause()); return null; } @@ -1570,6 +1566,7 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S } } }).exceptionally(ex -> { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex.getCause()); return null; }); @@ -1744,6 +1741,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + " topic", topic, FutureUtil.getException(topicFuture)); executor().submit(() -> { persistentTopic.close().whenComplete((ignore, ex) -> { + topics.remove(topic, topicFuture); if (ex != null) { log.warn("[{}] Get an error when closing topic.", topic, ex); @@ -1760,6 +1758,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + " Removing topic from topics list {}, {}", topic, ex); executor().submit(() -> { persistentTopic.close().whenComplete((ignore, closeEx) -> { + topics.remove(topic, topicFuture); if (closeEx != null) { log.warn("[{}] Get an error when closing topic.", topic, closeEx); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 765727aeac319..b58f416ea1a57 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -19,12 +19,10 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -1434,13 +1432,6 @@ public void testCleanupTopic() throws Exception { // Ok } - final CompletableFuture> timedOutTopicFuture = topicFuture; - // timeout topic future should be removed from cache - retryStrategically((test) -> pulsar1.getBrokerService().getTopic(topicName, false) != timedOutTopicFuture, 5, - 1000); - - assertNotEquals(timedOutTopicFuture, pulsar1.getBrokerService().getTopics().get(topicName)); - try { Consumer consumer = client1.newConsumer().topic(topicName).subscriptionType(SubscriptionType.Shared) .subscriptionName("my-subscriber-name").subscribeAsync().get(100, TimeUnit.MILLISECONDS); @@ -1452,6 +1443,7 @@ public void testCleanupTopic() throws Exception { ManagedLedgerImpl ml = (ManagedLedgerImpl) mlFactory.open(topicMlName + "-2"); mlFuture.complete(ml); + // Re-create topic will success. Consumer consumer = client1.newConsumer().topic(topicName).subscriptionName("my-subscriber-name") .subscriptionType(SubscriptionType.Shared).subscribeAsync() .get(2 * topicLoadTimeoutSeconds, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index 7cd9da7574dbb..d6473efd788d8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.client.api; +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.List; @@ -27,6 +29,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; @@ -34,6 +37,9 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.TopicPolicyListener; +import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; +import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; +import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.compaction.CompactionServiceFactory; @@ -108,6 +114,68 @@ public void testNoOrphanTopicAfterCreateTimeout() throws Exception { pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); } + @Test + public void testCloseLedgerThatTopicAfterCreateTimeout() throws Exception { + // Make the topic loading timeout faster. + long originalTopicLoadTimeoutSeconds = pulsar.getConfig().getTopicLoadTimeoutSeconds(); + int topicLoadTimeoutSeconds = 1; + pulsar.getConfig().setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); + pulsar.getConfig().setBrokerDeduplicationEnabled(true); + pulsar.getConfig().setTransactionCoordinatorEnabled(true); + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp2"); + + // Mock message deduplication recovery speed topicLoadTimeoutSeconds + String mlPath = BrokerService.MANAGED_LEDGER_PATH_ZNODE + "/" + + TopicName.get(tpName).getPersistenceNamingEncoding() + "/" + DEDUPLICATION_CURSOR_NAME; + mockZooKeeper.delay(topicLoadTimeoutSeconds * 1000, (op, path) -> { + if (mlPath.equals(path)) { + log.info("Topic load timeout: " + path); + return true; + } + return false; + }); + + // First load topic will trigger timeout + // The first topic load will trigger a timeout. When the topic closes, it will call transactionBuffer.close. + // Here, we simulate a sleep to ensure that the ledger is not immediately closed. + TransactionBufferProvider mockTransactionBufferProvider = new TransactionBufferProvider() { + @Override + public TransactionBuffer newTransactionBuffer(Topic originTopic) { + return new TransactionBufferDisable(originTopic) { + @SneakyThrows + @Override + public CompletableFuture closeAsync() { + Thread.sleep(500); + return super.closeAsync(); + } + }; + } + }; + TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); + pulsar.setTransactionExecutorProvider(mockTransactionBufferProvider); + CompletableFuture> firstLoad = pulsar.getBrokerService().getTopic(tpName, true); + Awaitility.await().ignoreExceptions().atMost(5, TimeUnit.SECONDS) + .pollInterval(100, TimeUnit.MILLISECONDS) + // assert first create topic timeout + .untilAsserted(() -> { + assertTrue(firstLoad.isCompletedExceptionally()); + }); + + // Once the first load topic times out, immediately to load the topic again. + Producer producer = pulsarClient.newProducer().topic(tpName).create(); + for (int i = 0; i < 10; i++) { + MessageId send = producer.send("msg".getBytes()); + Thread.sleep(100); + assertNotNull(send); + } + + // set to back + pulsar.setTransactionExecutorProvider(originalTransactionBufferProvider); + pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); + pulsar.getConfig().setBrokerDeduplicationEnabled(false); + pulsar.getConfig().setTransactionCoordinatorEnabled(false); + } + @Test public void testNoOrphanTopicIfInitFailed() throws Exception { String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp");