Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1824,7 +1824,11 @@ public void createPersistentTopic0(TopicLoadingContext context) {
maxTopicsCheck.thenCompose(partitionedTopicMetadata -> validateTopicConsistency(topicName))
.thenCompose(__ -> isTopicAlreadyMigrated)
.thenCompose(__ -> getManagedLedgerConfig(topicName))
.thenAccept(managedLedgerConfig -> {
.thenCombine(pulsar().getNamespaceService().checkTopicExistsAsync(topicName).thenApply(n -> {
boolean found = n.isExists();
n.recycle();
return found;
}), (managedLedgerConfig, exists) -> {
if (isBrokerEntryMetadataEnabled() || isBrokerPayloadProcessorEnabled()) {
// init managedLedger interceptor
Set<BrokerEntryMetadataInterceptor> interceptors = new HashSet<>();
Expand Down Expand Up @@ -1861,8 +1865,10 @@ public void createPersistentTopic0(TopicLoadingContext context) {
});

if (createIfMissing) {
topicEventsDispatcher.notify(topic, TopicEvent.CREATE, EventStage.BEFORE);
topicEventsDispatcher.notifyOnCompletion(topicFuture, topic, TopicEvent.CREATE);
if (!exists) {
topicEventsDispatcher.notify(topic, TopicEvent.CREATE, EventStage.BEFORE);
topicEventsDispatcher.notifyOnCompletion(topicFuture, topic, TopicEvent.CREATE);
}
}
topicEventsDispatcher.notifyOnCompletion(loadFuture, topic, TopicEvent.LOAD);

Expand Down Expand Up @@ -1952,7 +1958,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) {
}
}
}, () -> isTopicNsOwnedByBrokerAsync(topicName), null);

return null;
}).exceptionally((exception) -> {
boolean migrationFailure = exception.getCause() instanceof TopicMigratedException;
String msg = migrationFailure ? "Topic is already migrated" :
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker;

import static org.assertj.core.api.Assertions.assertThat;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import com.google.common.collect.Sets;
Expand Down Expand Up @@ -265,14 +266,30 @@ private void createTopicAndVerifyEvents(String topicDomain, String topicTypePart
final String[] expectedEvents;
if (topicDomain.equalsIgnoreCase("persistent") || topicTypePartitioned.equals("partitioned")) {
if (topicTypePartitioned.equals("partitioned")) {
expectedEvents = new String[]{
"CREATE__BEFORE",
"CREATE__SUCCESS",
"LOAD__BEFORE",
"CREATE__BEFORE",
"CREATE__SUCCESS",
"LOAD__SUCCESS"
};
if (topicDomain.equalsIgnoreCase("persistent")) {
expectedEvents = new String[]{
"CREATE__BEFORE",
"CREATE__SUCCESS",
"LOAD__BEFORE",
"LOAD__SUCCESS"
};
} else {
// For non-persistent partitioned topic, only metadata is initially created;
// partitions are created when the client connects.
// PR #23680 currently records creation events at metadata creation,
// and the broker records them again when partitions are loaded,
// which can result in multiple events.
// Ideally, #23680 should not record the event here,
// because the topic is not fully created until the client connects.
expectedEvents = new String[]{
"CREATE__BEFORE",
"CREATE__SUCCESS",
"LOAD__BEFORE",
"CREATE__BEFORE",
"CREATE__SUCCESS",
"LOAD__SUCCESS",
};
}
} else {
expectedEvents = new String[]{
"LOAD__BEFORE",
Expand Down Expand Up @@ -308,6 +325,53 @@ private void createTopicAndVerifyEvents(String topicDomain, String topicTypePart
Assert.assertEquals(events.toArray(), expectedEvents));
}

@DataProvider(name = "createTopicEventType")
public static Object[][] createTopicEventType() {
return new Object[][] {
{"persistent", "partitioned"},
{"persistent", "non-partitioned"},
{"non-persistent", "partitioned"},
{"non-persistent", "non-partitioned"},
};
}

@Test(dataProvider = "createTopicEventType")
public void testCreateTopicEvent(String topicTypePersistence, String topicTypePartitioned) throws Exception {
String topicName = topicTypePersistence + "://" + namespace + "/" + "topic-" + UUID.randomUUID();

events.clear();
if (topicTypePartitioned.equals("partitioned")) {
topicNameToWatch = topicName + "-partition-0";
admin.topics().createPartitionedTopic(topicName, 1);
} else {
topicNameToWatch = topicName;
admin.topics().createNonPartitionedTopic(topicName);
}

triggerPartitionsCreation(topicName); // ensure partitions are really created
triggerPartitionsCreation(topicName); // trigger again to ensure no duplicate events

Awaitility.await().during(3, TimeUnit.SECONDS)
.untilAsserted(() -> {
if (topicTypePartitioned.equals("partitioned") && topicTypePersistence.equals("non-persistent")) {
// For non-persistent partitioned topic, only metadata is initially created;
// partitions are created when the client connects.
// PR #23680 currently records creation events at metadata creation,
// and the broker records them again when partitions are loaded,
// which can result in multiple events.
// Ideally, #23680 should not record the event here,
// because the topic is not fully created until the client connects.
assertThat(events.toArray())
.contains("CREATE__BEFORE")
.contains("CREATE__SUCCESS");
} else {
assertThat(events.toArray())
.containsOnlyOnce("CREATE__BEFORE")
.containsOnlyOnce("CREATE__SUCCESS");
}
});
}

private PulsarAdmin createPulsarAdmin() throws PulsarClientException {
return PulsarAdmin.builder()
.serviceHttpUrl(brokerUrl != null ? brokerUrl.toString() : brokerUrlTls.toString())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.namespace.TopicExistsInfo;
import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator;
import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
Expand Down Expand Up @@ -226,6 +227,8 @@ public void setup() throws Exception {
doReturn(true).when(nsSvc).isServiceUnitOwned(any());
doReturn(CompletableFuture.completedFuture(mock(NamespaceBundle.class))).when(nsSvc).getBundleAsync(any());
doReturn(CompletableFuture.completedFuture(true)).when(nsSvc).checkBundleOwnership(any(), any());
doReturn(CompletableFuture.completedFuture(TopicExistsInfo.newTopicNotExists())).when(nsSvc)
.checkTopicExistsAsync(any());

setupMLAsyncCallbackMocks();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,7 @@
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.namespace.TopicExistsInfo;
import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
import org.apache.pulsar.broker.service.ServerCnx.State;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
Expand Down Expand Up @@ -237,6 +238,8 @@ public void setup() throws Exception {
NamespaceName.get("use", "ns-abc"), CommandGetTopicsOfNamespace.Mode.ALL);
doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfPersistentTopics(
NamespaceName.get("use", "ns-abc"));
doReturn(CompletableFuture.completedFuture(TopicExistsInfo.newTopicNotExists())).when(namespaceService)
.checkTopicExistsAsync(any());

setupMLAsyncCallbackMocks();

Expand Down