From 2aff9b39e3f2f21e885fa2d5addd04141d00eec1 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Mon, 27 Apr 2026 23:30:32 +0800
Subject: [PATCH 01/18] [feat][broker] PIP-469: Legacy-aware topic policies
backend routing and metadata-store topic policies
---
.../apache/pulsar/broker/PulsarService.java | 11 +-
.../LegacyAwareTopicPoliciesService.java | 179 ++++++++++++
.../MetadataStoreTopicPoliciesService.java | 275 ++++++++++++++++++
.../SystemTopicBasedTopicPoliciesService.java | 2 +-
.../LegacyAwareTopicPoliciesServiceTest.java | 163 +++++++++++
...MetadataStoreTopicPoliciesServiceTest.java | 168 +++++++++++
6 files changed, 796 insertions(+), 2 deletions(-)
create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
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 85bd9f53b1d2c..e8ccba688eef0 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
@@ -107,6 +107,7 @@
import org.apache.pulsar.broker.rest.Topics;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.HealthChecker;
+import org.apache.pulsar.broker.service.LegacyAwareTopicPoliciesService;
import org.apache.pulsar.broker.service.PulsarMetadataEventSynchronizer;
import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService;
import org.apache.pulsar.broker.service.Topic;
@@ -2288,8 +2289,16 @@ private TopicPoliciesService initTopicPoliciesService() throws Exception {
return TopicPoliciesService.DISABLED;
}
}
- return (TopicPoliciesService) Reflections.createInstance(className,
+ TopicPoliciesService configuredService = (TopicPoliciesService) Reflections.createInstance(className,
Thread.currentThread().getContextClassLoader());
+ if (!config.isSystemTopicEnabled()) {
+ log.warn()
+ .attr("className", className)
+ .log("System topic is disabled, using configured topic policies service without legacy routing");
+ return configuredService;
+ }
+ return new LegacyAwareTopicPoliciesService(this, new SystemTopicBasedTopicPoliciesService(this),
+ configuredService);
}
/**
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
new file mode 100644
index 0000000000000..b8e483f611773
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import lombok.CustomLog;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener;
+import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.NamespaceBundle;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.TopicPolicies;
+
+/**
+ * Routes topic policy operations to the legacy system-topic backend when a namespace already has
+ * a topic-policy {@code __change_events} system topic, and otherwise to the configured backend.
+ */
+@CustomLog
+public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
+
+ private final PulsarService pulsar;
+ private final TopicPoliciesService systemTopicService;
+ private final TopicPoliciesService configuredService;
+ private final Function> legacyNamespaceChecker;
+ private final Consumer systemTopicBundleLoad;
+ private final Consumer systemTopicBundleUnload;
+ private final Set ownedBundles = ConcurrentHashMap.newKeySet();
+ private final Set legacyOwnedBundles = ConcurrentHashMap.newKeySet();
+
+ public LegacyAwareTopicPoliciesService(PulsarService pulsar,
+ SystemTopicBasedTopicPoliciesService systemTopicService,
+ TopicPoliciesService configuredService) {
+ this(pulsar, systemTopicService, configuredService,
+ namespace -> NamespaceEventsSystemTopicFactory.checkSystemTopicExists(namespace,
+ EventType.TOPIC_POLICY, pulsar),
+ systemTopicService::addOwnedNamespaceBundleAsync,
+ systemTopicService::removeOwnedNamespaceBundleAsync);
+ }
+
+ LegacyAwareTopicPoliciesService(PulsarService pulsar,
+ TopicPoliciesService systemTopicService,
+ TopicPoliciesService configuredService,
+ Function> legacyNamespaceChecker,
+ Consumer systemTopicBundleLoad,
+ Consumer systemTopicBundleUnload) {
+ this.pulsar = pulsar;
+ this.systemTopicService = systemTopicService;
+ this.configuredService = configuredService;
+ this.legacyNamespaceChecker = legacyNamespaceChecker;
+ this.systemTopicBundleLoad = systemTopicBundleLoad;
+ this.systemTopicBundleUnload = systemTopicBundleUnload;
+ }
+
+ @Override
+ public void start(PulsarService pulsarService) {
+ configuredService.start(pulsarService);
+ pulsarService.getNamespaceService().addNamespaceBundleOwnershipListener(
+ new NamespaceBundleOwnershipListener() {
+ @Override
+ public void onLoad(NamespaceBundle bundle) {
+ pulsarService.getOrderedExecutor().executeOrdered(bundle.getNamespaceObject(),
+ () -> onBundleLoaded(bundle));
+ }
+
+ @Override
+ public void unLoad(NamespaceBundle bundle) {
+ pulsarService.getOrderedExecutor().executeOrdered(bundle.getNamespaceObject(),
+ () -> onBundleUnloaded(bundle));
+ }
+
+ @Override
+ public boolean test(NamespaceBundle namespaceBundle) {
+ return true;
+ }
+ });
+ }
+
+ @Override
+ public void close() throws Exception {
+ try {
+ configuredService.close();
+ } finally {
+ systemTopicService.close();
+ }
+ }
+
+ @Override
+ public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) {
+ return resolveService(topicName.getNamespaceObject())
+ .thenCompose(service -> service.getTopicPoliciesAsync(topicName, type));
+ }
+
+ @Override
+ public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boolean isGlobalPolicy,
+ boolean skipUpdateWhenTopicPolicyDoesntExist,
+ Consumer policyUpdater) {
+ return resolveService(topicName.getNamespaceObject())
+ .thenCompose(service -> service.updateTopicPoliciesAsync(topicName, isGlobalPolicy,
+ skipUpdateWhenTopicPolicyDoesntExist, policyUpdater));
+ }
+
+ @Override
+ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) {
+ return resolveService(topicName.getNamespaceObject())
+ .thenCompose(service -> service.deleteTopicPoliciesAsync(topicName));
+ }
+
+ @Override
+ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName,
+ boolean keepGlobalPoliciesAfterDeleting) {
+ return resolveService(topicName.getNamespaceObject())
+ .thenCompose(service -> service.deleteTopicPoliciesAsync(topicName,
+ keepGlobalPoliciesAfterDeleting));
+ }
+
+ @Override
+ public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
+ boolean configuredRegistered = configuredService.registerListener(topicName, listener);
+ boolean systemTopicRegistered = systemTopicService.registerListener(topicName, listener);
+ return configuredRegistered || systemTopicRegistered;
+ }
+
+ @Override
+ public void unregisterListener(TopicName topicName, TopicPolicyListener listener) {
+ configuredService.unregisterListener(topicName, listener);
+ systemTopicService.unregisterListener(topicName, listener);
+ }
+
+ CompletableFuture resolveService(NamespaceName namespace) {
+ return legacyNamespaceChecker.apply(namespace)
+ .thenApply(isLegacy -> Boolean.TRUE.equals(isLegacy) ? systemTopicService : configuredService);
+ }
+
+ void onBundleLoaded(NamespaceBundle bundle) {
+ ownedBundles.add(bundle);
+ legacyNamespaceChecker.apply(bundle.getNamespaceObject()).whenComplete((isLegacy, error) -> {
+ if (error != null) {
+ log.warn()
+ .attr("namespace", bundle.getNamespaceObject())
+ .exception(error)
+ .log("Failed to check topic-policy system topic for namespace");
+ return;
+ }
+ if (Boolean.TRUE.equals(isLegacy) && ownedBundles.contains(bundle) && legacyOwnedBundles.add(bundle)) {
+ systemTopicBundleLoad.accept(bundle);
+ }
+ });
+ }
+
+ void onBundleUnloaded(NamespaceBundle bundle) {
+ ownedBundles.remove(bundle);
+ if (legacyOwnedBundles.remove(bundle)) {
+ systemTopicBundleUnload.accept(bundle);
+ }
+ }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
new file mode 100644
index 0000000000000..c0bb06b8d4b0b
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import static java.util.Objects.requireNonNull;
+import com.google.common.annotations.VisibleForTesting;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import lombok.CustomLog;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.namespace.NamespaceService;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.TopicPolicies;
+import org.apache.pulsar.common.util.Codec;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.metadata.api.MetadataCache;
+import org.apache.pulsar.metadata.api.MetadataStore;
+import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException;
+import org.apache.pulsar.metadata.api.Notification;
+import org.apache.pulsar.metadata.api.NotificationType;
+
+/**
+ * Topic policies service backed by Pulsar metadata stores.
+ */
+@CustomLog
+public class MetadataStoreTopicPoliciesService implements TopicPoliciesService {
+
+ @VisibleForTesting
+ static final String GLOBAL_POLICIES_ROOT = "/admin/topic-policies";
+
+ @VisibleForTesting
+ static final String LOCAL_POLICIES_ROOT = "/admin/local-policies/topic-policies";
+
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+ private final Map> listeners = new ConcurrentHashMap<>();
+ private MetadataStore localStore;
+ private MetadataStore configurationStore;
+ private MetadataCache localPoliciesCache;
+ private MetadataCache globalPoliciesCache;
+
+ @Override
+ public void start(PulsarService pulsar) {
+ this.localStore = pulsar.getLocalMetadataStore();
+ this.configurationStore = pulsar.getConfigurationMetadataStore();
+ this.localPoliciesCache = localStore.getMetadataCache(TopicPolicies.class);
+ this.globalPoliciesCache = configurationStore.getMetadataCache(TopicPolicies.class);
+ this.localStore.registerListener(notification -> handleNotification(notification, false));
+ this.configurationStore.registerListener(notification -> handleNotification(notification, true));
+ }
+
+ @Override
+ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) {
+ return deleteTopicPoliciesAsync(topicName, false);
+ }
+
+ @Override
+ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName,
+ boolean keepGlobalPoliciesAfterDeleting) {
+ TopicName partitionedTopicName = normalizeTopicName(topicName);
+ if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
+ return CompletableFuture.completedFuture(null);
+ }
+ CompletableFuture deleteLocal =
+ deleteIfExists(localPoliciesCache, pathFor(partitionedTopicName, false));
+ if (keepGlobalPoliciesAfterDeleting) {
+ return deleteLocal;
+ }
+ CompletableFuture deleteGlobal =
+ deleteIfExists(globalPoliciesCache, pathFor(partitionedTopicName, true));
+ return CompletableFuture.allOf(deleteLocal, deleteGlobal);
+ }
+
+ @Override
+ public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boolean isGlobalPolicy,
+ boolean skipUpdateWhenTopicPolicyDoesntExist,
+ Consumer policyUpdater) {
+ requireNonNull(topicName);
+ requireNonNull(policyUpdater);
+ TopicName partitionedTopicName = normalizeTopicName(topicName);
+ if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
+ return CompletableFuture.failedFuture(new BrokerServiceException.NotAllowedException(
+ "Not allowed to update topic policy for the heartbeat topic"));
+ }
+ MetadataCache cache = cache(isGlobalPolicy);
+ String path = pathFor(partitionedTopicName, isGlobalPolicy);
+ CompletableFuture updateFuture;
+ if (skipUpdateWhenTopicPolicyDoesntExist) {
+ updateFuture = cache.readModifyUpdate(path,
+ current -> updatePolicies(Optional.of(current), isGlobalPolicy, policyUpdater));
+ } else {
+ updateFuture = cache.readModifyUpdateOrCreate(path,
+ current -> updatePolicies(current, isGlobalPolicy, policyUpdater));
+ }
+ return updateFuture.thenAccept(__ -> { }).exceptionally(error -> {
+ if (skipUpdateWhenTopicPolicyDoesntExist
+ && FutureUtil.unwrapCompletionException(error) instanceof NotFoundException) {
+ return null;
+ }
+ throw FutureUtil.wrapToCompletionException(error);
+ });
+ }
+
+ @Override
+ public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) {
+ requireNonNull(topicName);
+ TopicName partitionedTopicName = normalizeTopicName(topicName);
+ if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
+ return CompletableFuture.completedFuture(Optional.empty());
+ }
+ boolean global = type == GetType.GLOBAL_ONLY;
+ return cache(global).get(pathFor(partitionedTopicName, global))
+ .thenApply(policies -> policies.map(policy -> cloneWithScope(policy, global)));
+ }
+
+ @Override
+ public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
+ requireNonNull(topicName);
+ requireNonNull(listener);
+ listeners.compute(normalizeTopicName(topicName), (__, topicListeners) -> {
+ if (topicListeners == null) {
+ topicListeners = new CopyOnWriteArrayList<>();
+ }
+ topicListeners.add(listener);
+ return topicListeners;
+ });
+ return true;
+ }
+
+ @Override
+ public void unregisterListener(TopicName topicName, TopicPolicyListener listener) {
+ requireNonNull(topicName);
+ requireNonNull(listener);
+ listeners.computeIfPresent(normalizeTopicName(topicName), (__, topicListeners) -> {
+ topicListeners.remove(listener);
+ return topicListeners.isEmpty() ? null : topicListeners;
+ });
+ }
+
+ @Override
+ public void close() {
+ if (closed.compareAndSet(false, true)) {
+ listeners.clear();
+ if (localPoliciesCache != null) {
+ localPoliciesCache.invalidateAll();
+ }
+ if (globalPoliciesCache != null) {
+ globalPoliciesCache.invalidateAll();
+ }
+ }
+ }
+
+ private MetadataCache cache(boolean isGlobalPolicy) {
+ return isGlobalPolicy ? globalPoliciesCache : localPoliciesCache;
+ }
+
+ private CompletableFuture deleteIfExists(MetadataCache cache, String path) {
+ return cache.delete(path).handle((__, error) -> {
+ cache.invalidate(path);
+ if (error == null || FutureUtil.unwrapCompletionException(error) instanceof NotFoundException) {
+ return null;
+ }
+ throw FutureUtil.wrapToCompletionException(error);
+ });
+ }
+
+ private static TopicPolicies updatePolicies(Optional currentPolicies,
+ boolean isGlobalPolicy,
+ Consumer policyUpdater) {
+ TopicPolicies policies = currentPolicies.map(TopicPolicies::clone).orElseGet(TopicPolicies::new);
+ policies.setIsGlobal(isGlobalPolicy);
+ policyUpdater.accept(policies);
+ policies.setIsGlobal(isGlobalPolicy);
+ return policies;
+ }
+
+ private void handleNotification(Notification notification, boolean isGlobalPolicy) {
+ if (closed.get()
+ || (notification.getType() != NotificationType.Created
+ && notification.getType() != NotificationType.Modified
+ && notification.getType() != NotificationType.Deleted)) {
+ return;
+ }
+ String path = notification.getPath();
+ String root = isGlobalPolicy ? GLOBAL_POLICIES_ROOT : LOCAL_POLICIES_ROOT;
+ Optional topicName = topicNameFromPath(root, path);
+ if (topicName.isEmpty()) {
+ return;
+ }
+ MetadataCache cache = cache(isGlobalPolicy);
+ cache.invalidate(path);
+ if (notification.getType() == NotificationType.Deleted) {
+ notifyListeners(topicName.get(), null);
+ return;
+ }
+ cache.get(path).whenComplete((policies, error) -> {
+ if (error != null) {
+ log.warn()
+ .attr("path", path)
+ .exception(error)
+ .log("Failed to refresh topic policies after metadata notification");
+ return;
+ }
+ notifyListeners(topicName.get(),
+ policies.map(policy -> cloneWithScope(policy, isGlobalPolicy)).orElse(null));
+ });
+ }
+
+ private void notifyListeners(TopicName topicName, TopicPolicies policies) {
+ List topicListeners = listeners.get(topicName);
+ if (topicListeners == null) {
+ return;
+ }
+ for (TopicPolicyListener listener : topicListeners) {
+ try {
+ listener.onUpdate(policies == null ? null : policies.clone());
+ } catch (Throwable error) {
+ log.error().attr("topic", topicName).exception(error).log("Call topic policy listener error");
+ }
+ }
+ }
+
+ private static TopicName normalizeTopicName(TopicName topicName) {
+ return TopicName.get(topicName.getPartitionedTopicName());
+ }
+
+ private static TopicPolicies cloneWithScope(TopicPolicies policies, boolean isGlobalPolicy) {
+ TopicPolicies cloned = policies.clone();
+ cloned.setIsGlobal(isGlobalPolicy);
+ return cloned;
+ }
+
+ @VisibleForTesting
+ static String pathFor(TopicName topicName, boolean isGlobalPolicy) {
+ TopicName partitionedTopicName = normalizeTopicName(topicName);
+ return (isGlobalPolicy ? GLOBAL_POLICIES_ROOT : LOCAL_POLICIES_ROOT)
+ + "/" + partitionedTopicName.getTenant()
+ + "/" + partitionedTopicName.getNamespacePortion()
+ + "/" + partitionedTopicName.getDomain()
+ + "/" + partitionedTopicName.getEncodedLocalName();
+ }
+
+ @VisibleForTesting
+ static Optional topicNameFromPath(String root, String path) {
+ if (!path.startsWith(root + "/")) {
+ return Optional.empty();
+ }
+ String[] parts = path.substring(root.length() + 1).split("/", 4);
+ if (parts.length != 4) {
+ return Optional.empty();
+ }
+ return Optional.of(TopicName.get(parts[2], parts[0], parts[1], Codec.decode(parts[3])));
+ }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java
index 48653883d1a6a..a8f37d0c3894b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java
@@ -667,7 +667,7 @@ protected CompletableFuture> createSystemT
return systemTopicClient.newReaderAsync();
}
- private void removeOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) {
+ void removeOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) {
NamespaceName namespace = namespaceBundle.getNamespaceObject();
if (NamespaceService.isHeartbeatNamespace(namespace)) {
return;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
new file mode 100644
index 0000000000000..0473ed9af4cba
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertThrows;
+import static org.testng.Assert.assertTrue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import org.apache.pulsar.common.naming.NamespaceBundle;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.TopicPolicies;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker")
+public class LegacyAwareTopicPoliciesServiceTest {
+
+ @Test
+ public void testRoutesOperationsByLegacyMarker() throws Exception {
+ NamespaceName legacyNamespace = NamespaceName.get("tenant", "legacy");
+ RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
+ RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
+ LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
+ configuredService, namespace -> CompletableFuture.completedFuture(namespace.equals(legacyNamespace)),
+ __ -> { }, __ -> { });
+
+ service.updateTopicPoliciesAsync(TopicName.get("persistent://tenant/legacy/topic"), false, false,
+ policies -> { }).get();
+ service.getTopicPoliciesAsync(TopicName.get("persistent://tenant/new/topic"),
+ TopicPoliciesService.GetType.LOCAL_ONLY).get();
+ service.deleteTopicPoliciesAsync(TopicName.get("persistent://tenant/legacy/topic")).get();
+
+ assertEquals(systemTopicService.updateCount.get(), 1);
+ assertEquals(systemTopicService.deleteCount.get(), 1);
+ assertEquals(configuredService.getCount.get(), 1);
+ }
+
+ @Test
+ public void testLegacyCheckFailureDoesNotRouteToConfiguredBackend() {
+ RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
+ RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
+ LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
+ configuredService, __ -> CompletableFuture.failedFuture(new RuntimeException("failed marker check")),
+ __ -> { }, __ -> { });
+
+ assertThrows(ExecutionException.class, () -> service.getTopicPoliciesAsync(
+ TopicName.get("persistent://tenant/ns/topic"), TopicPoliciesService.GetType.LOCAL_ONLY).get());
+ assertEquals(systemTopicService.getCount.get(), 0);
+ assertEquals(configuredService.getCount.get(), 0);
+ }
+
+ @Test
+ public void testLoadsSystemTopicBackendOnlyForStillOwnedLegacyBundles() {
+ NamespaceName namespace = NamespaceName.get("tenant", "ns");
+ NamespaceBundle bundle = mock(NamespaceBundle.class);
+ when(bundle.getNamespaceObject()).thenReturn(namespace);
+ CompletableFuture markerCheck = new CompletableFuture<>();
+ List loads = new ArrayList<>();
+ List unloads = new ArrayList<>();
+ LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null,
+ new RecordingTopicPoliciesService(), new RecordingTopicPoliciesService(), __ -> markerCheck,
+ loads::add, unloads::add);
+
+ // Bundle loaded then unloaded before marker check completes
+ service.onBundleLoaded(bundle);
+ service.onBundleUnloaded(bundle);
+ markerCheck.complete(true);
+ assertTrue(loads.isEmpty());
+ assertTrue(unloads.isEmpty());
+
+ // Bundle loaded with synchronous marker check
+ CompletableFuture secondMarkerCheck = CompletableFuture.completedFuture(true);
+ service = new LegacyAwareTopicPoliciesService(null, new RecordingTopicPoliciesService(),
+ new RecordingTopicPoliciesService(), __ -> secondMarkerCheck, loads::add, unloads::add);
+ service.onBundleLoaded(bundle);
+ assertEquals(loads, List.of(bundle));
+ service.onBundleUnloaded(bundle);
+ assertEquals(unloads, List.of(bundle));
+ }
+
+ @Test
+ public void testRegisterAndUnregisterListenerOnBothBackends() {
+ RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
+ RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
+ LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
+ configuredService, __ -> CompletableFuture.completedFuture(true),
+ __ -> { }, __ -> { });
+
+ TopicName topic = TopicName.get("persistent://tenant/ns/topic");
+ TopicPolicyListener listener = policies -> { };
+
+ assertTrue(service.registerListener(topic, listener));
+ assertEquals(systemTopicService.registerListenerCount.get(), 1);
+ assertEquals(configuredService.registerListenerCount.get(), 1);
+
+ service.unregisterListener(topic, listener);
+ assertEquals(systemTopicService.unregisterListenerCount.get(), 1);
+ assertEquals(configuredService.unregisterListenerCount.get(), 1);
+ }
+
+ private static class RecordingTopicPoliciesService implements TopicPoliciesService {
+ private final AtomicInteger getCount = new AtomicInteger();
+ private final AtomicInteger updateCount = new AtomicInteger();
+ private final AtomicInteger deleteCount = new AtomicInteger();
+ private final AtomicInteger registerListenerCount = new AtomicInteger();
+ private final AtomicInteger unregisterListenerCount = new AtomicInteger();
+
+ @Override
+ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) {
+ deleteCount.incrementAndGet();
+ return CompletableFuture.completedFuture(null);
+ }
+
+ @Override
+ public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boolean isGlobalPolicy,
+ boolean skipUpdateWhenTopicPolicyDoesntExist,
+ Consumer policyUpdater) {
+ updateCount.incrementAndGet();
+ return CompletableFuture.completedFuture(null);
+ }
+
+ @Override
+ public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) {
+ getCount.incrementAndGet();
+ return CompletableFuture.completedFuture(Optional.empty());
+ }
+
+ @Override
+ public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
+ registerListenerCount.incrementAndGet();
+ return true;
+ }
+
+ @Override
+ public void unregisterListener(TopicName topicName, TopicPolicyListener listener) {
+ unregisterListenerCount.incrementAndGet();
+ }
+ }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
new file mode 100644
index 0000000000000..3578d44028c71
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.TopicPolicies;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
+import org.awaitility.Awaitility;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker")
+public class MetadataStoreTopicPoliciesServiceTest {
+
+ private MetadataStoreExtended localStore;
+ private MetadataStoreExtended configurationStore;
+ private MetadataStoreTopicPoliciesService service;
+
+ @BeforeMethod(alwaysRun = true)
+ public void setup() throws Exception {
+ localStore = MetadataStoreExtended.create("memory:local-" + UUID.randomUUID(),
+ MetadataStoreConfig.builder().build());
+ configurationStore = MetadataStoreExtended.create("memory:configuration-" + UUID.randomUUID(),
+ MetadataStoreConfig.builder().build());
+ PulsarService pulsar = mock(PulsarService.class);
+ when(pulsar.getLocalMetadataStore()).thenReturn(localStore);
+ when(pulsar.getConfigurationMetadataStore()).thenReturn(configurationStore);
+ service = new MetadataStoreTopicPoliciesService();
+ service.start(pulsar);
+ }
+
+ @AfterMethod(alwaysRun = true)
+ public void cleanup() throws Exception {
+ if (service != null) {
+ service.close();
+ }
+ if (localStore != null) {
+ localStore.close();
+ }
+ if (configurationStore != null) {
+ configurationStore.close();
+ }
+ }
+
+ @Test
+ public void testLocalAndGlobalPoliciesUseSeparateStoresAndNormalizePartitions() throws Exception {
+ TopicName partition = TopicName.get("persistent://tenant/ns/topic-partition-0");
+ TopicName topic = TopicName.get("persistent://tenant/ns/topic");
+
+ service.updateTopicPoliciesAsync(partition, false, false,
+ policies -> policies.setMaxConsumerPerTopic(3)).get();
+ service.updateTopicPoliciesAsync(partition, true, false,
+ policies -> policies.setMessageTTLInSeconds(10)).get();
+
+ Optional localPolicies =
+ service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get();
+ Optional globalPolicies =
+ service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get();
+
+ assertTrue(localPolicies.isPresent());
+ assertFalse(localPolicies.get().isGlobalPolicies());
+ assertEquals(localPolicies.get().getMaxConsumerPerTopic(), Integer.valueOf(3));
+ assertTrue(globalPolicies.isPresent());
+ assertTrue(globalPolicies.get().isGlobalPolicies());
+ assertEquals(globalPolicies.get().getMessageTTLInSeconds(), Integer.valueOf(10));
+ assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topic, false)).get());
+ assertTrue(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topic, true)).get());
+ }
+
+ @Test
+ public void testDeleteIsIdempotentAndCanKeepGlobalPolicies() throws Exception {
+ TopicName topic = TopicName.get("persistent://tenant/ns/delete-topic");
+
+ service.updateTopicPoliciesAsync(topic, true, false,
+ policies -> policies.setMessageTTLInSeconds(10)).get();
+ service.deleteTopicPoliciesAsync(topic, false).get();
+ assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get().isEmpty());
+ // second delete should be idempotent
+ service.deleteTopicPoliciesAsync(topic, false).get();
+ assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get().isEmpty());
+
+ service.updateTopicPoliciesAsync(topic, false, false,
+ policies -> policies.setMaxConsumerPerTopic(3)).get();
+ service.updateTopicPoliciesAsync(topic, true, false,
+ policies -> policies.setMessageTTLInSeconds(20)).get();
+ service.deleteTopicPoliciesAsync(topic, true).get();
+
+ assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get().isEmpty());
+ Optional globalPolicies =
+ service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get();
+ assertTrue(globalPolicies.isPresent());
+ assertEquals(globalPolicies.get().getMessageTTLInSeconds(), Integer.valueOf(20));
+ }
+
+ @Test
+ public void testListenerReceivesMetadataUpdatesAndDeletes() throws Exception {
+ TopicName topic = TopicName.get("persistent://tenant/ns/listener-topic");
+ List updates = new CopyOnWriteArrayList<>();
+ service.registerListener(topic, updates::add);
+
+ service.updateTopicPoliciesAsync(topic, false, false,
+ policies -> policies.setMaxProducerPerTopic(2)).get();
+ Awaitility.await().untilAsserted(() -> {
+ assertFalse(updates.isEmpty());
+ assertEquals(updates.get(updates.size() - 1).getMaxProducerPerTopic(), Integer.valueOf(2));
+ });
+
+ service.deleteTopicPoliciesAsync(topic, true).get();
+ Awaitility.await().untilAsserted(() -> assertTrue(updates.contains(null)));
+ }
+
+ @Test
+ public void testSkipUpdateWhenTopicPolicyDoesntExist() throws Exception {
+ TopicName topic = TopicName.get("persistent://tenant/ns/skip-update-topic");
+ // Should not throw when skip=true and policy doesn't exist
+ service.updateTopicPoliciesAsync(topic, false, true,
+ policies -> policies.setMaxConsumerPerTopic(5)).get();
+ assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get().isEmpty());
+
+ // Normal update creates the policy
+ service.updateTopicPoliciesAsync(topic, false, false,
+ policies -> policies.setMaxConsumerPerTopic(5)).get();
+ Optional result =
+ service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get();
+ assertTrue(result.isPresent());
+ assertEquals(result.get().getMaxConsumerPerTopic(), Integer.valueOf(5));
+ }
+
+ @Test
+ public void testPathFor() {
+ TopicName topic = TopicName.get("persistent://tenant/ns/topic");
+ String globalPath = MetadataStoreTopicPoliciesService.pathFor(topic, true);
+ String localPath = MetadataStoreTopicPoliciesService.pathFor(topic, false);
+
+ assertTrue(globalPath.startsWith(MetadataStoreTopicPoliciesService.GLOBAL_POLICIES_ROOT));
+ assertTrue(localPath.startsWith(MetadataStoreTopicPoliciesService.LOCAL_POLICIES_ROOT));
+ assertTrue(globalPath.contains("tenant/ns/"));
+ assertTrue(localPath.contains("tenant/ns/"));
+ }
+}
From 30a6220d7e232991eb9c54173196d522e9f405b1 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 10:43:57 +0800
Subject: [PATCH 02/18] fix: avoid duplicated listener if local and
configuration metadata stores are same
---
.../service/MetadataStoreTopicPoliciesService.java | 13 +++++++------
1 file changed, 7 insertions(+), 6 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index c0bb06b8d4b0b..a7019832ba5e8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -55,19 +55,20 @@ public class MetadataStoreTopicPoliciesService implements TopicPoliciesService {
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Map> listeners = new ConcurrentHashMap<>();
- private MetadataStore localStore;
- private MetadataStore configurationStore;
private MetadataCache localPoliciesCache;
private MetadataCache globalPoliciesCache;
@Override
public void start(PulsarService pulsar) {
- this.localStore = pulsar.getLocalMetadataStore();
- this.configurationStore = pulsar.getConfigurationMetadataStore();
+ MetadataStore localStore = pulsar.getLocalMetadataStore();
+ MetadataStore configurationStore = pulsar.getConfigurationMetadataStore();
this.localPoliciesCache = localStore.getMetadataCache(TopicPolicies.class);
this.globalPoliciesCache = configurationStore.getMetadataCache(TopicPolicies.class);
- this.localStore.registerListener(notification -> handleNotification(notification, false));
- this.configurationStore.registerListener(notification -> handleNotification(notification, true));
+ localStore.registerListener(notification -> handleNotification(notification, false));
+ if (localStore != configurationStore) {
+ // They are the same when geo-replication is not enabled, no need to register the same listener again
+ configurationStore.registerListener(notification -> handleNotification(notification, true));
+ }
}
@Override
From 7530092fefbeebbc34acf7109be700abac298448 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 11:26:35 +0800
Subject: [PATCH 03/18] fix: address comments
---
.../LegacyAwareTopicPoliciesService.java | 96 ++++++++++++++++---
.../MetadataStoreTopicPoliciesService.java | 9 ++
.../LegacyAwareTopicPoliciesServiceTest.java | 27 ++++++
...MetadataStoreTopicPoliciesServiceTest.java | 32 +++++++
4 files changed, 153 insertions(+), 11 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index b8e483f611773..c4592f43d8a64 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.service;
+import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
@@ -47,8 +48,8 @@ public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
private final Function> legacyNamespaceChecker;
private final Consumer systemTopicBundleLoad;
private final Consumer systemTopicBundleUnload;
- private final Set ownedBundles = ConcurrentHashMap.newKeySet();
- private final Set legacyOwnedBundles = ConcurrentHashMap.newKeySet();
+ private final Map> ownedBundles = new ConcurrentHashMap<>();
+ private final Map> legacyOwnedBundles = new ConcurrentHashMap<>();
public LegacyAwareTopicPoliciesService(PulsarService pulsar,
SystemTopicBasedTopicPoliciesService systemTopicService,
@@ -151,29 +152,102 @@ public void unregisterListener(TopicName topicName, TopicPolicyListener listener
CompletableFuture resolveService(NamespaceName namespace) {
return legacyNamespaceChecker.apply(namespace)
- .thenApply(isLegacy -> Boolean.TRUE.equals(isLegacy) ? systemTopicService : configuredService);
+ .thenCompose(isLegacy -> reconcileLegacyOwnership(namespace, Boolean.TRUE.equals(isLegacy))
+ .thenApply(__ -> Boolean.TRUE.equals(isLegacy) ? systemTopicService : configuredService));
}
void onBundleLoaded(NamespaceBundle bundle) {
- ownedBundles.add(bundle);
- legacyNamespaceChecker.apply(bundle.getNamespaceObject()).whenComplete((isLegacy, error) -> {
+ NamespaceName namespace = bundle.getNamespaceObject();
+ addBundle(ownedBundles, bundle);
+ legacyNamespaceChecker.apply(namespace).whenComplete((isLegacy, error) -> {
if (error != null) {
log.warn()
- .attr("namespace", bundle.getNamespaceObject())
+ .attr("namespace", namespace)
.exception(error)
.log("Failed to check topic-policy system topic for namespace");
return;
}
- if (Boolean.TRUE.equals(isLegacy) && ownedBundles.contains(bundle) && legacyOwnedBundles.add(bundle)) {
- systemTopicBundleLoad.accept(bundle);
- }
+ reconcileLegacyOwnership(namespace, Boolean.TRUE.equals(isLegacy)).exceptionally(reconcileError -> {
+ log.warn()
+ .attr("namespace", namespace)
+ .exception(reconcileError)
+ .log("Failed to reconcile legacy topic-policy ownership for namespace");
+ return null;
+ });
});
}
void onBundleUnloaded(NamespaceBundle bundle) {
- ownedBundles.remove(bundle);
- if (legacyOwnedBundles.remove(bundle)) {
+ removeBundle(ownedBundles, bundle);
+ if (removeBundle(legacyOwnedBundles, bundle)) {
+ systemTopicBundleUnload.accept(bundle);
+ }
+ }
+
+ private CompletableFuture reconcileLegacyOwnership(NamespaceName namespace, boolean isLegacy) {
+ return executeOrdered(namespace, () -> applyLegacyOwnership(namespace, isLegacy));
+ }
+
+ private CompletableFuture executeOrdered(NamespaceName namespace, Runnable action) {
+ if (pulsar == null) {
+ action.run();
+ return CompletableFuture.completedFuture(null);
+ }
+ CompletableFuture future = new CompletableFuture<>();
+ try {
+ pulsar.getOrderedExecutor().executeOrdered(namespace, () -> {
+ try {
+ action.run();
+ future.complete(null);
+ } catch (Throwable error) {
+ future.completeExceptionally(error);
+ }
+ });
+ } catch (Throwable error) {
+ future.completeExceptionally(error);
+ }
+ return future;
+ }
+
+ private void applyLegacyOwnership(NamespaceName namespace, boolean isLegacy) {
+ if (isLegacy) {
+ Set namespaceOwnedBundles = ownedBundles.get(namespace);
+ if (namespaceOwnedBundles == null || namespaceOwnedBundles.isEmpty()) {
+ return;
+ }
+ Set namespaceLegacyBundles =
+ legacyOwnedBundles.computeIfAbsent(namespace, __ -> ConcurrentHashMap.newKeySet());
+ for (NamespaceBundle bundle : namespaceOwnedBundles) {
+ if (namespaceLegacyBundles.add(bundle)) {
+ systemTopicBundleLoad.accept(bundle);
+ }
+ }
+ return;
+ }
+ Set namespaceLegacyBundles = legacyOwnedBundles.remove(namespace);
+ if (namespaceLegacyBundles == null) {
+ return;
+ }
+ for (NamespaceBundle bundle : namespaceLegacyBundles) {
systemTopicBundleUnload.accept(bundle);
}
}
+
+ private static void addBundle(Map> bundlesByNamespace, NamespaceBundle bundle) {
+ bundlesByNamespace.computeIfAbsent(bundle.getNamespaceObject(), __ -> ConcurrentHashMap.newKeySet()).add(bundle);
+ }
+
+ private static boolean removeBundle(Map> bundlesByNamespace,
+ NamespaceBundle bundle) {
+ NamespaceName namespace = bundle.getNamespaceObject();
+ Set namespaceBundles = bundlesByNamespace.get(namespace);
+ if (namespaceBundles == null) {
+ return false;
+ }
+ boolean removed = namespaceBundles.remove(bundle);
+ if (namespaceBundles.isEmpty()) {
+ bundlesByNamespace.remove(namespace, namespaceBundles);
+ }
+ return removed;
+ }
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index a7019832ba5e8..c46f715991e27 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -83,6 +83,9 @@ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName,
if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
return CompletableFuture.completedFuture(null);
}
+ if (closed.get()) {
+ return CompletableFuture.failedFuture(new BrokerServiceException(getClass().getName() + " is closed."));
+ }
CompletableFuture deleteLocal =
deleteIfExists(localPoliciesCache, pathFor(partitionedTopicName, false));
if (keepGlobalPoliciesAfterDeleting) {
@@ -104,6 +107,9 @@ public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boo
return CompletableFuture.failedFuture(new BrokerServiceException.NotAllowedException(
"Not allowed to update topic policy for the heartbeat topic"));
}
+ if (closed.get()) {
+ return CompletableFuture.failedFuture(new BrokerServiceException(getClass().getName() + " is closed."));
+ }
MetadataCache cache = cache(isGlobalPolicy);
String path = pathFor(partitionedTopicName, isGlobalPolicy);
CompletableFuture updateFuture;
@@ -130,6 +136,9 @@ public CompletableFuture> getTopicPoliciesAsync(TopicNam
if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
return CompletableFuture.completedFuture(Optional.empty());
}
+ if (closed.get()) {
+ return CompletableFuture.completedFuture(Optional.empty());
+ }
boolean global = type == GetType.GLOBAL_ONLY;
return cache(global).get(pathFor(partitionedTopicName, global))
.thenApply(policies -> policies.map(policy -> cloneWithScope(policy, global)));
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index 0473ed9af4cba..21afa6ab269d6 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -28,6 +28,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import org.apache.pulsar.common.naming.NamespaceBundle;
@@ -102,6 +103,32 @@ public void testLoadsSystemTopicBackendOnlyForStillOwnedLegacyBundles() {
assertEquals(unloads, List.of(bundle));
}
+ @Test
+ public void testResolveServiceReconcilesOwnedBundlesWhenLegacyStatusFlips() throws Exception {
+ NamespaceName namespace = NamespaceName.get("tenant", "ns");
+ NamespaceBundle bundle = mock(NamespaceBundle.class);
+ when(bundle.getNamespaceObject()).thenReturn(namespace);
+ AtomicBoolean isLegacy = new AtomicBoolean(false);
+ List loads = new ArrayList<>();
+ List unloads = new ArrayList<>();
+ RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
+ RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
+ LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
+ configuredService, __ -> CompletableFuture.completedFuture(isLegacy.get()), loads::add, unloads::add);
+
+ service.onBundleLoaded(bundle);
+ assertEquals(service.resolveService(namespace).get(), configuredService);
+ assertTrue(loads.isEmpty());
+
+ isLegacy.set(true);
+ assertEquals(service.resolveService(namespace).get(), systemTopicService);
+ assertEquals(loads, List.of(bundle));
+
+ isLegacy.set(false);
+ assertEquals(service.resolveService(namespace).get(), configuredService);
+ assertEquals(unloads, List.of(bundle));
+ }
+
@Test
public void testRegisterAndUnregisterListenerOnBothBackends() {
RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
index 3578d44028c71..219a7086fef0f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
@@ -23,10 +23,12 @@
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
import java.util.List;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutionException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.TopicPolicies;
@@ -154,6 +156,36 @@ public void testSkipUpdateWhenTopicPolicyDoesntExist() throws Exception {
assertEquals(result.get().getMaxConsumerPerTopic(), Integer.valueOf(5));
}
+ @Test
+ public void testCloseStopsReadsAndWrites() throws Exception {
+ TopicName existingTopic = TopicName.get("persistent://tenant/ns/closed-topic");
+ TopicName newTopic = TopicName.get("persistent://tenant/ns/closed-topic-new");
+
+ service.updateTopicPoliciesAsync(existingTopic, false, false,
+ policies -> policies.setMaxConsumerPerTopic(7)).get();
+ service.close();
+
+ assertTrue(service.getTopicPoliciesAsync(existingTopic, TopicPoliciesService.GetType.LOCAL_ONLY)
+ .get().isEmpty());
+
+ try {
+ service.updateTopicPoliciesAsync(newTopic, false, false,
+ policies -> policies.setMaxConsumerPerTopic(9)).get();
+ fail("Expected update after close to fail");
+ } catch (ExecutionException error) {
+ assertTrue(error.getCause() instanceof BrokerServiceException);
+ }
+ assertFalse(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(newTopic, false)).get());
+
+ try {
+ service.deleteTopicPoliciesAsync(existingTopic).get();
+ fail("Expected delete after close to fail");
+ } catch (ExecutionException error) {
+ assertTrue(error.getCause() instanceof BrokerServiceException);
+ }
+ assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(existingTopic, false)).get());
+ }
+
@Test
public void testPathFor() {
TopicName topic = TopicName.get("persistent://tenant/ns/topic");
From 8e13d2e9445c17549b2f6873fba8a33f0247780d Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 12:39:09 +0800
Subject: [PATCH 04/18] Test initializations of various topic policies services
---
.../service/TopicPoliciesServiceInitTest.java | 101 ++++++++++++++++++
1 file changed, 101 insertions(+)
create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java
new file mode 100644
index 0000000000000..4349a1181dac8
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pulsar.broker.service;
+
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import java.util.Optional;
+import lombok.Cleanup;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker")
+public class TopicPoliciesServiceInitTest {
+
+ private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(1, 0, () -> 0);
+ private ServiceConfiguration config;
+
+ @BeforeClass
+ public void setup() throws Exception {
+ bk.start();
+ }
+
+ @AfterClass
+ public void teardown() throws Exception {
+ bk.stop();
+ }
+
+ @BeforeMethod
+ public void initConfig() {
+ config = new ServiceConfiguration();
+ config.setClusterName("testCluster");
+ config.setMetadataStoreUrl("zk:127.0.0.1:" + bk.getZookeeperPort());
+ config.setWebServicePort(Optional.of(0));
+ config.setBrokerServicePort(Optional.of(0));
+ config.setAdvertisedAddress("127.0.0.1");
+ config.setManagedLedgerDefaultWriteQuorum(1);
+ config.setManagedLedgerDefaultAckQuorum(1);
+ config.setManagedLedgerDefaultEnsembleSize(1);
+ }
+
+ @Test
+ public void testSystemTopicPoliciesService() throws Exception {
+ config.setTopicPoliciesServiceClassName(SystemTopicBasedTopicPoliciesService.class.getName());
+ @Cleanup final var pulsar = new PulsarService(config);
+ pulsar.start();
+ assertTrue(pulsar.getTopicPoliciesService() instanceof SystemTopicBasedTopicPoliciesService);
+ }
+
+ @Test
+ public void testLegacyAwareTopicPoliciesService() throws Exception {
+ config.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
+ @Cleanup final var pulsar = new PulsarService(config);
+ pulsar.start();
+ assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
+ }
+
+ @Test
+ public void testMetadataStoreTopicPoliciesService() throws Exception {
+ config.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
+ // the topic policies service won't be aware of the legacy `__change_events` system topics
+ config.setSystemTopicEnabled(false);
+ @Cleanup final var pulsar = new PulsarService(config);
+ pulsar.start();
+ assertTrue(pulsar.getTopicPoliciesService() instanceof MetadataStoreTopicPoliciesService);
+ }
+
+ @Test
+ public void testWrongInitialization() throws Exception {
+ config.setTopicPoliciesServiceClassName(LegacyAwareTopicPoliciesService.class.getName());
+ @Cleanup final var pulsar = new PulsarService(config);
+ try {
+ pulsar.start();
+ fail();
+ } catch (PulsarServerException e) {
+ assertTrue(e.getCause().getCause() instanceof NoSuchMethodException);
+ }
+ }
+}
From 281772fb8ed9aa8b04b253e0c6f7de39de68b7dc Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 13:42:34 +0800
Subject: [PATCH 05/18] Change unit tests to integration tests
---
.../LegacyAwareTopicPoliciesServiceTest.java | 244 +++++++-----------
...MetadataStoreTopicPoliciesServiceTest.java | 230 ++++++-----------
2 files changed, 175 insertions(+), 299 deletions(-)
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index 21afa6ab269d6..e541b06d0a9f7 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -18,173 +18,121 @@
*/
package org.apache.pulsar.broker.service;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertThrows;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Consumer;
-import org.apache.pulsar.common.naming.NamespaceBundle;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
+import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
-import org.apache.pulsar.common.policies.data.TopicPolicies;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@Test(groups = "broker")
-public class LegacyAwareTopicPoliciesServiceTest {
-
- @Test
- public void testRoutesOperationsByLegacyMarker() throws Exception {
- NamespaceName legacyNamespace = NamespaceName.get("tenant", "legacy");
- RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
- RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
- LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
- configuredService, namespace -> CompletableFuture.completedFuture(namespace.equals(legacyNamespace)),
- __ -> { }, __ -> { });
-
- service.updateTopicPoliciesAsync(TopicName.get("persistent://tenant/legacy/topic"), false, false,
- policies -> { }).get();
- service.getTopicPoliciesAsync(TopicName.get("persistent://tenant/new/topic"),
- TopicPoliciesService.GetType.LOCAL_ONLY).get();
- service.deleteTopicPoliciesAsync(TopicName.get("persistent://tenant/legacy/topic")).get();
-
- assertEquals(systemTopicService.updateCount.get(), 1);
- assertEquals(systemTopicService.deleteCount.get(), 1);
- assertEquals(configuredService.getCount.get(), 1);
+public class LegacyAwareTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
+
+ @BeforeClass
+ @Override
+ protected void setup() throws Exception {
+ super.internalSetup();
+ super.setupDefaultTenantAndNamespace();
+ assertTrue(pulsar.getTopicPoliciesService() instanceof SystemTopicBasedTopicPoliciesService);
}
- @Test
- public void testLegacyCheckFailureDoesNotRouteToConfiguredBackend() {
- RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
- RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
- LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
- configuredService, __ -> CompletableFuture.failedFuture(new RuntimeException("failed marker check")),
- __ -> { }, __ -> { });
-
- assertThrows(ExecutionException.class, () -> service.getTopicPoliciesAsync(
- TopicName.get("persistent://tenant/ns/topic"), TopicPoliciesService.GetType.LOCAL_ONLY).get());
- assertEquals(systemTopicService.getCount.get(), 0);
- assertEquals(configuredService.getCount.get(), 0);
+ @AfterClass
+ @Override
+ protected void cleanup() throws Exception {
+ super.internalCleanup();
}
@Test
- public void testLoadsSystemTopicBackendOnlyForStillOwnedLegacyBundles() {
- NamespaceName namespace = NamespaceName.get("tenant", "ns");
- NamespaceBundle bundle = mock(NamespaceBundle.class);
- when(bundle.getNamespaceObject()).thenReturn(namespace);
- CompletableFuture markerCheck = new CompletableFuture<>();
- List loads = new ArrayList<>();
- List unloads = new ArrayList<>();
- LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null,
- new RecordingTopicPoliciesService(), new RecordingTopicPoliciesService(), __ -> markerCheck,
- loads::add, unloads::add);
-
- // Bundle loaded then unloaded before marker check completes
- service.onBundleLoaded(bundle);
- service.onBundleUnloaded(bundle);
- markerCheck.complete(true);
- assertTrue(loads.isEmpty());
- assertTrue(unloads.isEmpty());
-
- // Bundle loaded with synchronous marker check
- CompletableFuture secondMarkerCheck = CompletableFuture.completedFuture(true);
- service = new LegacyAwareTopicPoliciesService(null, new RecordingTopicPoliciesService(),
- new RecordingTopicPoliciesService(), __ -> secondMarkerCheck, loads::add, unloads::add);
- service.onBundleLoaded(bundle);
- assertEquals(loads, List.of(bundle));
- service.onBundleUnloaded(bundle);
- assertEquals(unloads, List.of(bundle));
- }
+ public void testLegacyNamespaceKeepsSystemTopicBackendAfterRestart() throws Exception {
+ final var namespace1 = "public/legacy-aware-ns1";
+ final var topic1 = "persistent://" + namespace1 + "/topic";
+ final var eventTopic1 = NamespaceEventsSystemTopicFactory
+ .getEventsTopicName(NamespaceName.get(namespace1))
+ .toString();
+ admin.namespaces().createNamespace(namespace1);
+ admin.topics().createNonPartitionedTopic(topic1);
+
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic1).create()) {
+ producer.send("warmup");
+ }
+ final var namespace1TopicBeforeRestart = getPersistentTopic(topic1);
- @Test
- public void testResolveServiceReconcilesOwnedBundlesWhenLegacyStatusFlips() throws Exception {
- NamespaceName namespace = NamespaceName.get("tenant", "ns");
- NamespaceBundle bundle = mock(NamespaceBundle.class);
- when(bundle.getNamespaceObject()).thenReturn(namespace);
- AtomicBoolean isLegacy = new AtomicBoolean(false);
- List loads = new ArrayList<>();
- List unloads = new ArrayList<>();
- RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
- RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
- LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
- configuredService, __ -> CompletableFuture.completedFuture(isLegacy.get()), loads::add, unloads::add);
-
- service.onBundleLoaded(bundle);
- assertEquals(service.resolveService(namespace).get(), configuredService);
- assertTrue(loads.isEmpty());
-
- isLegacy.set(true);
- assertEquals(service.resolveService(namespace).get(), systemTopicService);
- assertEquals(loads, List.of(bundle));
-
- isLegacy.set(false);
- assertEquals(service.resolveService(namespace).get(), configuredService);
- assertEquals(unloads, List.of(bundle));
- }
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic1));
- @Test
- public void testRegisterAndUnregisterListenerOnBothBackends() {
- RecordingTopicPoliciesService systemTopicService = new RecordingTopicPoliciesService();
- RecordingTopicPoliciesService configuredService = new RecordingTopicPoliciesService();
- LegacyAwareTopicPoliciesService service = new LegacyAwareTopicPoliciesService(null, systemTopicService,
- configuredService, __ -> CompletableFuture.completedFuture(true),
- __ -> { }, __ -> { });
-
- TopicName topic = TopicName.get("persistent://tenant/ns/topic");
- TopicPolicyListener listener = policies -> { };
-
- assertTrue(service.registerListener(topic, listener));
- assertEquals(systemTopicService.registerListenerCount.get(), 1);
- assertEquals(configuredService.registerListenerCount.get(), 1);
-
- service.unregisterListener(topic, listener);
- assertEquals(systemTopicService.unregisterListenerCount.get(), 1);
- assertEquals(configuredService.unregisterListenerCount.get(), 1);
- }
+ admin.topicPolicies().setCompactionThreshold(topic1, 1000);
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), Long.valueOf(1000));
+ assertEquals(namespace1TopicBeforeRestart.getHierarchyTopicPolicies().getCompactionThreshold().get(),
+ Long.valueOf(1000));
+ assertTrue(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic1))
+ .join());
+ });
- private static class RecordingTopicPoliciesService implements TopicPoliciesService {
- private final AtomicInteger getCount = new AtomicInteger();
- private final AtomicInteger updateCount = new AtomicInteger();
- private final AtomicInteger deleteCount = new AtomicInteger();
- private final AtomicInteger registerListenerCount = new AtomicInteger();
- private final AtomicInteger unregisterListenerCount = new AtomicInteger();
-
- @Override
- public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) {
- deleteCount.incrementAndGet();
- return CompletableFuture.completedFuture(null);
- }
+ restartBroker(configuration ->
+ configuration.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName()));
+ assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
- @Override
- public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boolean isGlobalPolicy,
- boolean skipUpdateWhenTopicPolicyDoesntExist,
- Consumer policyUpdater) {
- updateCount.incrementAndGet();
- return CompletableFuture.completedFuture(null);
- }
+ Awaitility.await().untilAsserted(() ->
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), Long.valueOf(1000)));
- @Override
- public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) {
- getCount.incrementAndGet();
- return CompletableFuture.completedFuture(Optional.empty());
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic1).create()) {
+ producer.send("after-restart");
}
-
- @Override
- public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
- registerListenerCount.incrementAndGet();
- return true;
+ final var namespace1TopicAfterRestart = getPersistentTopic(topic1);
+ final var namespace1LacBeforeUpdate = admin.topics().getInternalStats(eventTopic1).lastConfirmedEntry;
+
+ admin.topicPolicies().setCompactionThreshold(topic1, 2000);
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), Long.valueOf(2000));
+ assertEquals(namespace1TopicAfterRestart.getHierarchyTopicPolicies().getCompactionThreshold().get(),
+ Long.valueOf(2000));
+ Assert.assertNotEquals(admin.topics().getInternalStats(eventTopic1).lastConfirmedEntry,
+ namespace1LacBeforeUpdate);
+ });
+ assertFalse(pulsar.getLocalMetadataStore()
+ .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic1), false))
+ .join());
+
+ final var namespace2 = "public/legacy-aware-ns2";
+ final var topic2 = "persistent://" + namespace2 + "/topic";
+ final var eventTopic2 = NamespaceEventsSystemTopicFactory
+ .getEventsTopicName(NamespaceName.get(namespace2))
+ .toString();
+ admin.namespaces().createNamespace(namespace2);
+ admin.topics().createNonPartitionedTopic(topic2);
+
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic2).create()) {
+ producer.send("warmup");
}
+ final var namespace2Topic = getPersistentTopic(topic2);
+
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic2));
+ assertFalse(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic2))
+ .join());
+
+ admin.topicPolicies().setCompactionThreshold(topic2, 3000);
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic2), Long.valueOf(3000));
+ assertEquals(namespace2Topic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
+ Long.valueOf(3000));
+ assertFalse(pulsar.getPulsarResources().getTopicResources()
+ .persistentTopicExists(TopicName.get(eventTopic2)).join());
+ assertTrue(pulsar.getLocalMetadataStore()
+ .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic2), false))
+ .join());
+ });
+ }
- @Override
- public void unregisterListener(TopicName topicName, TopicPolicyListener listener) {
- unregisterListenerCount.incrementAndGet();
- }
+ private PersistentTopic getPersistentTopic(String topic) throws Exception {
+ return (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElseThrow();
}
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
index 219a7086fef0f..619e66f8e887f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
@@ -18,183 +18,111 @@
*/
package org.apache.pulsar.broker.service;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
-import static org.testng.Assert.fail;
import java.util.List;
-import java.util.Optional;
-import java.util.UUID;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutionException;
-import org.apache.pulsar.broker.PulsarService;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.naming.TopicName;
-import org.apache.pulsar.common.policies.data.TopicPolicies;
-import org.apache.pulsar.metadata.api.MetadataStoreConfig;
-import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
import org.awaitility.Awaitility;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@Test(groups = "broker")
-public class MetadataStoreTopicPoliciesServiceTest {
-
- private MetadataStoreExtended localStore;
- private MetadataStoreExtended configurationStore;
- private MetadataStoreTopicPoliciesService service;
-
- @BeforeMethod(alwaysRun = true)
- public void setup() throws Exception {
- localStore = MetadataStoreExtended.create("memory:local-" + UUID.randomUUID(),
- MetadataStoreConfig.builder().build());
- configurationStore = MetadataStoreExtended.create("memory:configuration-" + UUID.randomUUID(),
- MetadataStoreConfig.builder().build());
- PulsarService pulsar = mock(PulsarService.class);
- when(pulsar.getLocalMetadataStore()).thenReturn(localStore);
- when(pulsar.getConfigurationMetadataStore()).thenReturn(configurationStore);
- service = new MetadataStoreTopicPoliciesService();
- service.start(pulsar);
+public class MetadataStoreTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
+
+ @BeforeClass
+ @Override
+ protected void setup() throws Exception {
+ conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
+ conf.setSystemTopicEnabled(false);
+ super.internalSetup();
+ super.setupDefaultTenantAndNamespace();
+ assertTrue(pulsar.getTopicPoliciesService() instanceof MetadataStoreTopicPoliciesService);
}
- @AfterMethod(alwaysRun = true)
- public void cleanup() throws Exception {
- if (service != null) {
- service.close();
- }
- if (localStore != null) {
- localStore.close();
- }
- if (configurationStore != null) {
- configurationStore.close();
- }
+ @AfterClass
+ @Override
+ protected void cleanup() throws Exception {
+ super.internalCleanup();
}
@Test
- public void testLocalAndGlobalPoliciesUseSeparateStoresAndNormalizePartitions() throws Exception {
- TopicName partition = TopicName.get("persistent://tenant/ns/topic-partition-0");
- TopicName topic = TopicName.get("persistent://tenant/ns/topic");
-
- service.updateTopicPoliciesAsync(partition, false, false,
- policies -> policies.setMaxConsumerPerTopic(3)).get();
- service.updateTopicPoliciesAsync(partition, true, false,
- policies -> policies.setMessageTTLInSeconds(10)).get();
-
- Optional localPolicies =
- service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get();
- Optional globalPolicies =
- service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get();
-
- assertTrue(localPolicies.isPresent());
- assertFalse(localPolicies.get().isGlobalPolicies());
- assertEquals(localPolicies.get().getMaxConsumerPerTopic(), Integer.valueOf(3));
- assertTrue(globalPolicies.isPresent());
- assertTrue(globalPolicies.get().isGlobalPolicies());
- assertEquals(globalPolicies.get().getMessageTTLInSeconds(), Integer.valueOf(10));
- assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topic, false)).get());
- assertTrue(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topic, true)).get());
- }
+ public void testTopicPoliciesAdminPersistsAndUpdatesLoadedTopic() throws Exception {
+ final var topic = TopicName.get("test-metadata-store-topic-policies").toString();
+ admin.topics().createNonPartitionedTopic(topic);
- @Test
- public void testDeleteIsIdempotentAndCanKeepGlobalPolicies() throws Exception {
- TopicName topic = TopicName.get("persistent://tenant/ns/delete-topic");
-
- service.updateTopicPoliciesAsync(topic, true, false,
- policies -> policies.setMessageTTLInSeconds(10)).get();
- service.deleteTopicPoliciesAsync(topic, false).get();
- assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get().isEmpty());
- // second delete should be idempotent
- service.deleteTopicPoliciesAsync(topic, false).get();
- assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get().isEmpty());
-
- service.updateTopicPoliciesAsync(topic, false, false,
- policies -> policies.setMaxConsumerPerTopic(3)).get();
- service.updateTopicPoliciesAsync(topic, true, false,
- policies -> policies.setMessageTTLInSeconds(20)).get();
- service.deleteTopicPoliciesAsync(topic, true).get();
-
- assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get().isEmpty());
- Optional globalPolicies =
- service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.GLOBAL_ONLY).get();
- assertTrue(globalPolicies.isPresent());
- assertEquals(globalPolicies.get().getMessageTTLInSeconds(), Integer.valueOf(20));
- }
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create()) {
+ producer.send("warmup");
+ }
+ final var persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get()
+ .orElseThrow();
- @Test
- public void testListenerReceivesMetadataUpdatesAndDeletes() throws Exception {
- TopicName topic = TopicName.get("persistent://tenant/ns/listener-topic");
- List updates = new CopyOnWriteArrayList<>();
- service.registerListener(topic, updates::add);
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic));
+ Assert.assertNotEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
+ Long.valueOf(1000));
- service.updateTopicPoliciesAsync(topic, false, false,
- policies -> policies.setMaxProducerPerTopic(2)).get();
+ admin.topicPolicies().setCompactionThreshold(topic, 1000);
Awaitility.await().untilAsserted(() -> {
- assertFalse(updates.isEmpty());
- assertEquals(updates.get(updates.size() - 1).getMaxProducerPerTopic(), Integer.valueOf(2));
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000));
+ assertEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(), Long.valueOf(1000));
});
- service.deleteTopicPoliciesAsync(topic, true).get();
- Awaitility.await().untilAsserted(() -> assertTrue(updates.contains(null)));
+ restartBroker();
+
+ Awaitility.await().untilAsserted(() ->
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000)));
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create()) {
+ producer.send("after-restart");
+ }
+ final var reloadedTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get()
+ .orElseThrow();
+ Awaitility.await().untilAsserted(() ->
+ assertEquals(reloadedTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
+ Long.valueOf(1000)));
}
@Test
- public void testSkipUpdateWhenTopicPolicyDoesntExist() throws Exception {
- TopicName topic = TopicName.get("persistent://tenant/ns/skip-update-topic");
- // Should not throw when skip=true and policy doesn't exist
- service.updateTopicPoliciesAsync(topic, false, true,
- policies -> policies.setMaxConsumerPerTopic(5)).get();
- assertTrue(service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get().isEmpty());
-
- // Normal update creates the policy
- service.updateTopicPoliciesAsync(topic, false, false,
- policies -> policies.setMaxConsumerPerTopic(5)).get();
- Optional result =
- service.getTopicPoliciesAsync(topic, TopicPoliciesService.GetType.LOCAL_ONLY).get();
- assertTrue(result.isPresent());
- assertEquals(result.get().getMaxConsumerPerTopic(), Integer.valueOf(5));
+ public void testTopicPoliciesDeletedWithTopic() throws Exception {
+ final var topic = TopicName.get("test-metadata-store-topic-policies-delete").toString();
+ admin.topics().createNonPartitionedTopic(topic);
+
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic));
+ admin.topicPolicies().setCompactionThreshold(topic, 1000);
+ Awaitility.await().untilAsserted(() ->
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000)));
+
+ admin.topics().delete(topic);
+ admin.topics().createNonPartitionedTopic(topic);
+ Awaitility.await().untilAsserted(() -> assertNull(admin.topicPolicies().getCompactionThreshold(topic)));
}
@Test
- public void testCloseStopsReadsAndWrites() throws Exception {
- TopicName existingTopic = TopicName.get("persistent://tenant/ns/closed-topic");
- TopicName newTopic = TopicName.get("persistent://tenant/ns/closed-topic-new");
-
- service.updateTopicPoliciesAsync(existingTopic, false, false,
- policies -> policies.setMaxConsumerPerTopic(7)).get();
- service.close();
-
- assertTrue(service.getTopicPoliciesAsync(existingTopic, TopicPoliciesService.GetType.LOCAL_ONLY)
- .get().isEmpty());
-
- try {
- service.updateTopicPoliciesAsync(newTopic, false, false,
- policies -> policies.setMaxConsumerPerTopic(9)).get();
- fail("Expected update after close to fail");
- } catch (ExecutionException error) {
- assertTrue(error.getCause() instanceof BrokerServiceException);
+ public void testShadowReplicator() throws Exception {
+ final var sourceTopic = TopicName.get("test-metadata-shadow-replicator").toString();
+ final var shadowTopic = sourceTopic + "-shadow";
+
+ admin.topics().createNonPartitionedTopic(sourceTopic);
+ admin.topics().createShadowTopic(shadowTopic, sourceTopic);
+ admin.topics().setShadowTopics(sourceTopic, List.of(shadowTopic));
+
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create();
+ var consumer = pulsarClient.newConsumer(Schema.STRING).topic(shadowTopic)
+ .subscriptionName("sub").subscribe()) {
+ producer.send("msg");
+ final var msg = consumer.receive(5, TimeUnit.SECONDS);
+ Assert.assertNotNull(msg);
+ Assert.assertEquals(msg.getValue(), "msg");
}
- assertFalse(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(newTopic, false)).get());
- try {
- service.deleteTopicPoliciesAsync(existingTopic).get();
- fail("Expected delete after close to fail");
- } catch (ExecutionException error) {
- assertTrue(error.getCause() instanceof BrokerServiceException);
- }
- assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(existingTopic, false)).get());
- }
-
- @Test
- public void testPathFor() {
- TopicName topic = TopicName.get("persistent://tenant/ns/topic");
- String globalPath = MetadataStoreTopicPoliciesService.pathFor(topic, true);
- String localPath = MetadataStoreTopicPoliciesService.pathFor(topic, false);
-
- assertTrue(globalPath.startsWith(MetadataStoreTopicPoliciesService.GLOBAL_POLICIES_ROOT));
- assertTrue(localPath.startsWith(MetadataStoreTopicPoliciesService.LOCAL_POLICIES_ROOT));
- assertTrue(globalPath.contains("tenant/ns/"));
- assertTrue(localPath.contains("tenant/ns/"));
+ final var persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(sourceTopic).get()
+ .orElseThrow();
+ Awaitility.await().untilAsserted(() ->
+ Assert.assertEquals(persistentTopic.getShadowReplicators().size(), 1));
}
}
From 36afb62e5c784c5aa5d4ed0a4d9be9751212be87 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 15:04:47 +0800
Subject: [PATCH 06/18] chore: clean up unnecessary code
---
.../apache/pulsar/broker/PulsarService.java | 4 +-
.../LegacyAwareTopicPoliciesService.java | 56 ++++++-------------
2 files changed, 19 insertions(+), 41 deletions(-)
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 e8ccba688eef0..d6a47b4eb7cd1 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
@@ -2289,10 +2289,10 @@ private TopicPoliciesService initTopicPoliciesService() throws Exception {
return TopicPoliciesService.DISABLED;
}
}
- TopicPoliciesService configuredService = (TopicPoliciesService) Reflections.createInstance(className,
+ final var configuredService = (TopicPoliciesService) Reflections.createInstance(className,
Thread.currentThread().getContextClassLoader());
if (!config.isSystemTopicEnabled()) {
- log.warn()
+ log.info()
.attr("className", className)
.log("System topic is disabled, using configured topic policies service without legacy routing");
return configuredService;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index c4592f43d8a64..be19a8054c444 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -24,7 +24,6 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
-import java.util.function.Function;
import lombok.CustomLog;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener;
@@ -43,53 +42,33 @@
public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
private final PulsarService pulsar;
- private final TopicPoliciesService systemTopicService;
+ private final SystemTopicBasedTopicPoliciesService systemTopicService;
private final TopicPoliciesService configuredService;
- private final Function> legacyNamespaceChecker;
- private final Consumer systemTopicBundleLoad;
- private final Consumer systemTopicBundleUnload;
private final Map> ownedBundles = new ConcurrentHashMap<>();
private final Map> legacyOwnedBundles = new ConcurrentHashMap<>();
public LegacyAwareTopicPoliciesService(PulsarService pulsar,
SystemTopicBasedTopicPoliciesService systemTopicService,
TopicPoliciesService configuredService) {
- this(pulsar, systemTopicService, configuredService,
- namespace -> NamespaceEventsSystemTopicFactory.checkSystemTopicExists(namespace,
- EventType.TOPIC_POLICY, pulsar),
- systemTopicService::addOwnedNamespaceBundleAsync,
- systemTopicService::removeOwnedNamespaceBundleAsync);
- }
-
- LegacyAwareTopicPoliciesService(PulsarService pulsar,
- TopicPoliciesService systemTopicService,
- TopicPoliciesService configuredService,
- Function> legacyNamespaceChecker,
- Consumer systemTopicBundleLoad,
- Consumer systemTopicBundleUnload) {
this.pulsar = pulsar;
this.systemTopicService = systemTopicService;
this.configuredService = configuredService;
- this.legacyNamespaceChecker = legacyNamespaceChecker;
- this.systemTopicBundleLoad = systemTopicBundleLoad;
- this.systemTopicBundleUnload = systemTopicBundleUnload;
}
@Override
public void start(PulsarService pulsarService) {
configuredService.start(pulsarService);
+ final var self = this;
pulsarService.getNamespaceService().addNamespaceBundleOwnershipListener(
new NamespaceBundleOwnershipListener() {
@Override
public void onLoad(NamespaceBundle bundle) {
- pulsarService.getOrderedExecutor().executeOrdered(bundle.getNamespaceObject(),
- () -> onBundleLoaded(bundle));
+ self.executeOrdered(bundle.getNamespaceObject(), () -> onBundleLoaded(bundle));
}
@Override
public void unLoad(NamespaceBundle bundle) {
- pulsarService.getOrderedExecutor().executeOrdered(bundle.getNamespaceObject(),
- () -> onBundleUnloaded(bundle));
+ self.executeOrdered(bundle.getNamespaceObject(), () -> onBundleUnloaded(bundle));
}
@Override
@@ -150,16 +129,15 @@ public void unregisterListener(TopicName topicName, TopicPolicyListener listener
systemTopicService.unregisterListener(topicName, listener);
}
- CompletableFuture resolveService(NamespaceName namespace) {
- return legacyNamespaceChecker.apply(namespace)
- .thenCompose(isLegacy -> reconcileLegacyOwnership(namespace, Boolean.TRUE.equals(isLegacy))
- .thenApply(__ -> Boolean.TRUE.equals(isLegacy) ? systemTopicService : configuredService));
+ private CompletableFuture resolveService(NamespaceName namespace) {
+ return isLegacy(namespace).thenCompose(isLegacy -> reconcileLegacyOwnership(namespace, isLegacy)
+ .thenApply(__ -> isLegacy ? systemTopicService : configuredService));
}
- void onBundleLoaded(NamespaceBundle bundle) {
+ private void onBundleLoaded(NamespaceBundle bundle) {
NamespaceName namespace = bundle.getNamespaceObject();
addBundle(ownedBundles, bundle);
- legacyNamespaceChecker.apply(namespace).whenComplete((isLegacy, error) -> {
+ isLegacy(namespace).whenComplete((isLegacy, error) -> {
if (error != null) {
log.warn()
.attr("namespace", namespace)
@@ -177,10 +155,10 @@ void onBundleLoaded(NamespaceBundle bundle) {
});
}
- void onBundleUnloaded(NamespaceBundle bundle) {
+ private void onBundleUnloaded(NamespaceBundle bundle) {
removeBundle(ownedBundles, bundle);
if (removeBundle(legacyOwnedBundles, bundle)) {
- systemTopicBundleUnload.accept(bundle);
+ systemTopicService.removeOwnedNamespaceBundleAsync(bundle);
}
}
@@ -189,10 +167,6 @@ private CompletableFuture reconcileLegacyOwnership(NamespaceName namespace
}
private CompletableFuture executeOrdered(NamespaceName namespace, Runnable action) {
- if (pulsar == null) {
- action.run();
- return CompletableFuture.completedFuture(null);
- }
CompletableFuture future = new CompletableFuture<>();
try {
pulsar.getOrderedExecutor().executeOrdered(namespace, () -> {
@@ -219,7 +193,7 @@ private void applyLegacyOwnership(NamespaceName namespace, boolean isLegacy) {
legacyOwnedBundles.computeIfAbsent(namespace, __ -> ConcurrentHashMap.newKeySet());
for (NamespaceBundle bundle : namespaceOwnedBundles) {
if (namespaceLegacyBundles.add(bundle)) {
- systemTopicBundleLoad.accept(bundle);
+ systemTopicService.addOwnedNamespaceBundleAsync(bundle);
}
}
return;
@@ -229,7 +203,7 @@ private void applyLegacyOwnership(NamespaceName namespace, boolean isLegacy) {
return;
}
for (NamespaceBundle bundle : namespaceLegacyBundles) {
- systemTopicBundleUnload.accept(bundle);
+ systemTopicService.removeOwnedNamespaceBundleAsync(bundle);
}
}
@@ -250,4 +224,8 @@ private static boolean removeBundle(Map> bun
}
return removed;
}
+
+ private CompletableFuture isLegacy(NamespaceName namespace) {
+ return NamespaceEventsSystemTopicFactory.checkSystemTopicExists(namespace, EventType.TOPIC_POLICY, pulsar);
+ }
}
From e6b5d8fd531b580f594d22a8779cc5eb849f62ab Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 15:13:41 +0800
Subject: [PATCH 07/18] chore: remove unnecessary caches from
LegacyAwareTopicPoliciesService
---
.../LegacyAwareTopicPoliciesService.java | 119 +-----------------
.../LegacyAwareTopicPoliciesServiceTest.java | 58 +++++++++
2 files changed, 59 insertions(+), 118 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index be19a8054c444..6b472184f354d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -18,18 +18,13 @@
*/
package org.apache.pulsar.broker.service;
-import java.util.Map;
import java.util.Optional;
-import java.util.Set;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import lombok.CustomLog;
import org.apache.pulsar.broker.PulsarService;
-import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener;
import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
import org.apache.pulsar.common.events.EventType;
-import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.TopicPolicies;
@@ -44,8 +39,6 @@ public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
private final PulsarService pulsar;
private final SystemTopicBasedTopicPoliciesService systemTopicService;
private final TopicPoliciesService configuredService;
- private final Map> ownedBundles = new ConcurrentHashMap<>();
- private final Map> legacyOwnedBundles = new ConcurrentHashMap<>();
public LegacyAwareTopicPoliciesService(PulsarService pulsar,
SystemTopicBasedTopicPoliciesService systemTopicService,
@@ -58,24 +51,6 @@ public LegacyAwareTopicPoliciesService(PulsarService pulsar,
@Override
public void start(PulsarService pulsarService) {
configuredService.start(pulsarService);
- final var self = this;
- pulsarService.getNamespaceService().addNamespaceBundleOwnershipListener(
- new NamespaceBundleOwnershipListener() {
- @Override
- public void onLoad(NamespaceBundle bundle) {
- self.executeOrdered(bundle.getNamespaceObject(), () -> onBundleLoaded(bundle));
- }
-
- @Override
- public void unLoad(NamespaceBundle bundle) {
- self.executeOrdered(bundle.getNamespaceObject(), () -> onBundleUnloaded(bundle));
- }
-
- @Override
- public boolean test(NamespaceBundle namespaceBundle) {
- return true;
- }
- });
}
@Override
@@ -130,99 +105,7 @@ public void unregisterListener(TopicName topicName, TopicPolicyListener listener
}
private CompletableFuture resolveService(NamespaceName namespace) {
- return isLegacy(namespace).thenCompose(isLegacy -> reconcileLegacyOwnership(namespace, isLegacy)
- .thenApply(__ -> isLegacy ? systemTopicService : configuredService));
- }
-
- private void onBundleLoaded(NamespaceBundle bundle) {
- NamespaceName namespace = bundle.getNamespaceObject();
- addBundle(ownedBundles, bundle);
- isLegacy(namespace).whenComplete((isLegacy, error) -> {
- if (error != null) {
- log.warn()
- .attr("namespace", namespace)
- .exception(error)
- .log("Failed to check topic-policy system topic for namespace");
- return;
- }
- reconcileLegacyOwnership(namespace, Boolean.TRUE.equals(isLegacy)).exceptionally(reconcileError -> {
- log.warn()
- .attr("namespace", namespace)
- .exception(reconcileError)
- .log("Failed to reconcile legacy topic-policy ownership for namespace");
- return null;
- });
- });
- }
-
- private void onBundleUnloaded(NamespaceBundle bundle) {
- removeBundle(ownedBundles, bundle);
- if (removeBundle(legacyOwnedBundles, bundle)) {
- systemTopicService.removeOwnedNamespaceBundleAsync(bundle);
- }
- }
-
- private CompletableFuture reconcileLegacyOwnership(NamespaceName namespace, boolean isLegacy) {
- return executeOrdered(namespace, () -> applyLegacyOwnership(namespace, isLegacy));
- }
-
- private CompletableFuture executeOrdered(NamespaceName namespace, Runnable action) {
- CompletableFuture future = new CompletableFuture<>();
- try {
- pulsar.getOrderedExecutor().executeOrdered(namespace, () -> {
- try {
- action.run();
- future.complete(null);
- } catch (Throwable error) {
- future.completeExceptionally(error);
- }
- });
- } catch (Throwable error) {
- future.completeExceptionally(error);
- }
- return future;
- }
-
- private void applyLegacyOwnership(NamespaceName namespace, boolean isLegacy) {
- if (isLegacy) {
- Set namespaceOwnedBundles = ownedBundles.get(namespace);
- if (namespaceOwnedBundles == null || namespaceOwnedBundles.isEmpty()) {
- return;
- }
- Set namespaceLegacyBundles =
- legacyOwnedBundles.computeIfAbsent(namespace, __ -> ConcurrentHashMap.newKeySet());
- for (NamespaceBundle bundle : namespaceOwnedBundles) {
- if (namespaceLegacyBundles.add(bundle)) {
- systemTopicService.addOwnedNamespaceBundleAsync(bundle);
- }
- }
- return;
- }
- Set namespaceLegacyBundles = legacyOwnedBundles.remove(namespace);
- if (namespaceLegacyBundles == null) {
- return;
- }
- for (NamespaceBundle bundle : namespaceLegacyBundles) {
- systemTopicService.removeOwnedNamespaceBundleAsync(bundle);
- }
- }
-
- private static void addBundle(Map> bundlesByNamespace, NamespaceBundle bundle) {
- bundlesByNamespace.computeIfAbsent(bundle.getNamespaceObject(), __ -> ConcurrentHashMap.newKeySet()).add(bundle);
- }
-
- private static boolean removeBundle(Map> bundlesByNamespace,
- NamespaceBundle bundle) {
- NamespaceName namespace = bundle.getNamespaceObject();
- Set namespaceBundles = bundlesByNamespace.get(namespace);
- if (namespaceBundles == null) {
- return false;
- }
- boolean removed = namespaceBundles.remove(bundle);
- if (namespaceBundles.isEmpty()) {
- bundlesByNamespace.remove(namespace, namespaceBundles);
- }
- return removed;
+ return isLegacy(namespace).thenApply(isLegacy -> isLegacy ? systemTopicService : configuredService);
}
private CompletableFuture isLegacy(NamespaceName namespace) {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index e541b06d0a9f7..aef4c923fb55d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -53,6 +53,8 @@ protected void cleanup() throws Exception {
@Test
public void testLegacyNamespaceKeepsSystemTopicBackendAfterRestart() throws Exception {
+ restartWithSystemTopicPoliciesService();
+
final var namespace1 = "public/legacy-aware-ns1";
final var topic1 = "persistent://" + namespace1 + "/topic";
final var eventTopic1 = NamespaceEventsSystemTopicFactory
@@ -132,6 +134,62 @@ public void testLegacyNamespaceKeepsSystemTopicBackendAfterRestart() throws Exce
});
}
+ @Test
+ public void testOwnedBundleCanSwitchToLegacyBackendAfterNamespaceBecomesLegacy() throws Exception {
+ restartWithLegacyAwareMetadataStoreService();
+
+ final var namespace = "public/legacy-aware-flip";
+ final var topic = "persistent://" + namespace + "/topic";
+ final var eventTopic = NamespaceEventsSystemTopicFactory
+ .getEventsTopicName(NamespaceName.get(namespace))
+ .toString();
+ admin.namespaces().createNamespace(namespace);
+ admin.topics().createNonPartitionedTopic(topic);
+
+ try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create()) {
+ producer.send("warmup");
+ }
+ final var persistentTopic = getPersistentTopic(topic);
+
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic));
+ assertFalse(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic))
+ .join());
+ assertFalse(pulsar.getLocalMetadataStore()
+ .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic), false))
+ .join());
+
+ admin.topics().createNonPartitionedTopic(eventTopic);
+ Awaitility.await().untilAsserted(() ->
+ assertTrue(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic))
+ .join()));
+
+ final var eventTopicLastConfirmedEntryBeforeUpdate = admin.topics().getInternalStats(eventTopic)
+ .lastConfirmedEntry;
+ admin.topicPolicies().setCompactionThreshold(topic, 1000);
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000));
+ assertEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
+ Long.valueOf(1000));
+ Assert.assertNotEquals(admin.topics().getInternalStats(eventTopic).lastConfirmedEntry,
+ eventTopicLastConfirmedEntryBeforeUpdate);
+ assertFalse(pulsar.getLocalMetadataStore()
+ .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic), false))
+ .join());
+ });
+ }
+
+ private void restartWithSystemTopicPoliciesService() throws Exception {
+ restartBroker(configuration ->
+ configuration.setTopicPoliciesServiceClassName(SystemTopicBasedTopicPoliciesService.class.getName()));
+ assertTrue(pulsar.getTopicPoliciesService() instanceof SystemTopicBasedTopicPoliciesService);
+ }
+
+ private void restartWithLegacyAwareMetadataStoreService() throws Exception {
+ restartBroker(configuration ->
+ configuration.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName()));
+ assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
+ }
+
private PersistentTopic getPersistentTopic(String topic) throws Exception {
return (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElseThrow();
}
From 6148a0ffc5b3e8807b666461d6d06ccac7f6b628 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 15:26:39 +0800
Subject: [PATCH 08/18] refactor: add integration tests for both cases whether
system topic is enabled
---
.../LegacyAwareTopicPoliciesServiceTest.java | 15 +++------------
.../MetadataStoreTopicPoliciesServiceTest.java | 12 ++++++++++--
2 files changed, 13 insertions(+), 14 deletions(-)
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index aef4c923fb55d..e22b89731e072 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -35,20 +35,11 @@
import org.testng.annotations.Test;
@Test(groups = "broker")
-public class LegacyAwareTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
+public class LegacyAwareTopicPoliciesServiceTest extends MetadataStoreTopicPoliciesServiceTest{
- @BeforeClass
@Override
- protected void setup() throws Exception {
- super.internalSetup();
- super.setupDefaultTenantAndNamespace();
- assertTrue(pulsar.getTopicPoliciesService() instanceof SystemTopicBasedTopicPoliciesService);
- }
-
- @AfterClass
- @Override
- protected void cleanup() throws Exception {
- super.internalCleanup();
+ protected boolean isLegacyTopicPoliciesService() {
+ return true;
}
@Test
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
index 619e66f8e887f..b1049a7e40508 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
@@ -36,14 +36,22 @@
@Test(groups = "broker")
public class MetadataStoreTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
+ protected boolean isLegacyTopicPoliciesService() {
+ return false;
+ }
+
@BeforeClass
@Override
protected void setup() throws Exception {
conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
- conf.setSystemTopicEnabled(false);
+ conf.setSystemTopicEnabled(isLegacyTopicPoliciesService());
super.internalSetup();
super.setupDefaultTenantAndNamespace();
- assertTrue(pulsar.getTopicPoliciesService() instanceof MetadataStoreTopicPoliciesService);
+ if (isLegacyTopicPoliciesService()) {
+ assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
+ } else {
+ assertTrue(pulsar.getTopicPoliciesService() instanceof MetadataStoreTopicPoliciesService);
+ }
}
@AfterClass
From f98f95ac43cc0dee4212d501f91369e1935da047 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 19:58:19 +0800
Subject: [PATCH 09/18] fix: error logs from the metadata listener of
NamespaceBundleFactory
---
.../MetadataStoreTopicPoliciesService.java | 9 +++-----
...MetadataStoreTopicPoliciesServiceTest.java | 23 +++++++++++++++++++
2 files changed, 26 insertions(+), 6 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index c46f715991e27..938b6ddca20fb 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -47,11 +47,8 @@
@CustomLog
public class MetadataStoreTopicPoliciesService implements TopicPoliciesService {
- @VisibleForTesting
- static final String GLOBAL_POLICIES_ROOT = "/admin/topic-policies";
-
- @VisibleForTesting
- static final String LOCAL_POLICIES_ROOT = "/admin/local-policies/topic-policies";
+ private static final String GLOBAL_POLICIES_ROOT = "/admin/topic-policies/global";
+ private static final String LOCAL_POLICIES_ROOT = "/admin/topic-policies/local";
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Map> listeners = new ConcurrentHashMap<>();
@@ -272,7 +269,7 @@ static String pathFor(TopicName topicName, boolean isGlobalPolicy) {
}
@VisibleForTesting
- static Optional topicNameFromPath(String root, String path) {
+ private static Optional topicNameFromPath(String root, String path) {
if (!path.startsWith(root + "/")) {
return Optional.empty();
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
index b1049a7e40508..d570848076c19 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
@@ -24,9 +24,11 @@
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.utils.TestLogAppender;
import org.awaitility.Awaitility;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
@@ -110,6 +112,27 @@ public void testTopicPoliciesDeletedWithTopic() throws Exception {
Awaitility.await().untilAsserted(() -> assertNull(admin.topicPolicies().getCompactionThreshold(topic)));
}
+ // Verify the topic policies metadata path does not have conflicts with "/admin/local-policies", which is registered
+ // with a listener from `NamespaceBundleFactory`.
+ @Test
+ public void testLocalTopicPoliciesPathDoesNotTriggerNamespaceBundleFactoryError() throws Exception {
+ final var topic = TopicName.get("test-metadata-store-topic-policies-log-regression").toString();
+ admin.topics().createNonPartitionedTopic(topic);
+
+ try (var appender = TestLogAppender.create(NamespaceBundleFactory.class)) {
+ appender.clearEvents();
+
+ admin.topicPolicies().setCompactionThreshold(topic, 1000);
+ Awaitility.await().untilAsserted(() ->
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000)));
+
+ Awaitility.await().during(1, TimeUnit.SECONDS).atMost(2, TimeUnit.SECONDS).until(() ->
+ appender.getEvents().stream().noneMatch(event ->
+ event.getMessage().getFormattedMessage()
+ .contains("Failed to update the policy change for path")));
+ }
+ }
+
@Test
public void testShadowReplicator() throws Exception {
final var sourceTopic = TopicName.get("test-metadata-shadow-replicator").toString();
From bd3adb65b11b4e9c916f50e57a9313f6bddae582 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 21:02:33 +0800
Subject: [PATCH 10/18] fix: global policies listeners are not called
---
.../MetadataStoreTopicPoliciesService.java | 8 ++-
.../LegacyAwareTopicPoliciesServiceTest.java | 8 +--
...MetadataStoreTopicPoliciesServiceTest.java | 51 +++++++++++++++++--
3 files changed, 53 insertions(+), 14 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index 938b6ddca20fb..d6b6a94fef59b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -40,6 +40,7 @@
import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;
+import org.jspecify.annotations.Nullable;
/**
* Topic policies service backed by Pulsar metadata stores.
@@ -62,10 +63,7 @@ public void start(PulsarService pulsar) {
this.localPoliciesCache = localStore.getMetadataCache(TopicPolicies.class);
this.globalPoliciesCache = configurationStore.getMetadataCache(TopicPolicies.class);
localStore.registerListener(notification -> handleNotification(notification, false));
- if (localStore != configurationStore) {
- // They are the same when geo-replication is not enabled, no need to register the same listener again
- configurationStore.registerListener(notification -> handleNotification(notification, true));
- }
+ configurationStore.registerListener(notification -> handleNotification(notification, true));
}
@Override
@@ -234,7 +232,7 @@ private void handleNotification(Notification notification, boolean isGlobalPolic
});
}
- private void notifyListeners(TopicName topicName, TopicPolicies policies) {
+ private void notifyListeners(TopicName topicName, @Nullable TopicPolicies policies) {
List topicListeners = listeners.get(topicName);
if (topicListeners == null) {
return;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index e22b89731e072..539e215479abb 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -22,7 +22,6 @@
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
-import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
import org.apache.pulsar.client.api.Schema;
@@ -30,8 +29,6 @@
import org.apache.pulsar.common.naming.TopicName;
import org.awaitility.Awaitility;
import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@Test(groups = "broker")
@@ -150,9 +147,8 @@ public void testOwnedBundleCanSwitchToLegacyBackendAfterNamespaceBecomesLegacy()
.join());
admin.topics().createNonPartitionedTopic(eventTopic);
- Awaitility.await().untilAsserted(() ->
- assertTrue(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic))
- .join()));
+ Awaitility.await().untilAsserted(() -> assertTrue(pulsar.getPulsarResources().getTopicResources()
+ .persistentTopicExists(TopicName.get(eventTopic)).join()));
final var eventTopicLastConfirmedEntryBeforeUpdate = admin.topics().getInternalStats(eventTopic)
.lastConfirmedEntry;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
index d570848076c19..c2719ab7eb324 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
@@ -21,13 +21,18 @@
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
+import java.time.Duration;
import java.util.List;
+import java.util.Optional;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
-import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.testcontext.PulsarTestContext;
import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.apache.pulsar.utils.TestLogAppender;
import org.awaitility.Awaitility;
import org.testng.Assert;
@@ -42,6 +47,11 @@ protected boolean isLegacyTopicPoliciesService() {
return false;
}
+ @Override
+ protected void configureMetadataStores(PulsarTestContext.Builder builder) {
+ builder.withMockZookeeper();
+ }
+
@BeforeClass
@Override
protected void setup() throws Exception {
@@ -79,8 +89,8 @@ public void testTopicPoliciesAdminPersistsAndUpdatesLoadedTopic() throws Excepti
admin.topicPolicies().setCompactionThreshold(topic, 1000);
Awaitility.await().untilAsserted(() -> {
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000));
- assertEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(), Long.valueOf(1000));
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), 1000L);
+ assertEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(), 1000L);
});
restartBroker();
@@ -156,4 +166,39 @@ public void testShadowReplicator() throws Exception {
Awaitility.await().untilAsserted(() ->
Assert.assertEquals(persistentTopic.getShadowReplicators().size(), 1));
}
+
+ @Test
+ public void testListenerTriggered() throws Exception {
+ final var topic = TopicName.get("test-global-policies-not-triggered").toString();
+ final var topicName = TopicName.get(topic);
+ admin.topics().createNonPartitionedTopic(topic);
+
+ final var compactionThreshold = new AtomicLong(0);
+ pulsar.getTopicPoliciesService().registerListener(topicName, policies ->
+ Optional.ofNullable(policies).map(TopicPolicies::getCompactionThreshold).ifPresentOrElse(
+ compactionThreshold::set, () -> compactionThreshold.set(-1)));
+
+ // Verify Created events are handled
+ admin.topicPolicies(false).setCompactionThreshold(topic, 100);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 100));
+
+ admin.topicPolicies(true).setCompactionThreshold(topic, 200);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 200));
+
+ // Verify Modified events are handled
+ admin.topicPolicies(false).setCompactionThreshold(topic, 300);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 300));
+
+ admin.topicPolicies(true).setCompactionThreshold(topic, 400);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 400));
+
+ // Verify Deleted events are handled
+ admin.topicPolicies(false).deleteTopicPolicies(topic);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), -1));
+ }
}
From d1fc536367078791c63b6ddb8633b83315f8872b Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Wed, 6 May 2026 21:55:06 +0800
Subject: [PATCH 11/18] chore: speed up TopicPoliciesTest
current: BUILD SUCCESSFUL in 2m 18s
---
.../broker/admin/TopicPoliciesTest.java | 72 ++++++++++++++-----
1 file changed, 56 insertions(+), 16 deletions(-)
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
index d94aa51b73bfe..92d2e4561ae4b 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
@@ -119,7 +119,9 @@
import org.glassfish.jersey.client.JerseyClientBuilder;
import org.mockito.Mockito;
import org.testng.Assert;
+import org.testng.annotations.AfterClass;
import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
@@ -144,10 +146,11 @@ public class TopicPoliciesTest extends MockedPulsarServiceBaseTest {
private final int testTopicPartitions = 2;
- @BeforeMethod
+ @BeforeClass(alwaysRun = true)
@Override
protected void setup() throws Exception {
this.conf.setDefaultNumberOfNamespaceBundles(1);
+ this.conf.setForceDeleteNamespaceAllowed(true);
super.internalSetup();
admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
@@ -156,17 +159,34 @@ protected void setup() throws Exception {
admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Set.of("test"));
admin.namespaces().createNamespace(myNamespaceV1);
admin.topics().createPartitionedTopic(testTopic, testTopicPartitions);
- Producer> producer = pulsarClient.newProducer().topic(testTopic).create();
- producer.close();
- waitForZooKeeperWatchers();
}
- @AfterMethod(alwaysRun = true)
+ @AfterClass(alwaysRun = true)
@Override
public void cleanup() throws Exception {
super.internalCleanup();
}
+ @BeforeMethod
+ void setupTestTopic() throws Exception {
+ // Recreate namespace to clear any policies set by previous tests
+ try {
+ admin.topics().deletePartitionedTopic(testTopic, true);
+ } catch (PulsarAdminException.NotFoundException e) {
+ // topic may already be deleted
+ }
+ admin.namespaces().deleteNamespace(myNamespace, true);
+ admin.namespaces().deleteNamespace(myNamespaceV1, true);
+ admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Set.of("test"));
+ admin.namespaces().createNamespace(myNamespaceV1);
+ admin.topics().createPartitionedTopic(testTopic, testTopicPartitions);
+ }
+
+ @AfterMethod
+ void afterMethodCleanup() {
+ clearTopicPoliciesCache();
+ }
+
@Test
public void updatePropertiesForAutoCreatedTopicTest() throws Exception {
TopicName topicName = TopicName.get(
@@ -2756,10 +2776,8 @@ public void testRemoveSubscribeRate() throws Exception {
@Test
public void testPublishRateInDifferentLevelPolicy() throws Exception {
- cleanup();
- conf.setMaxPublishRatePerTopicInMessages(5);
- conf.setMaxPublishRatePerTopicInBytes(50L);
- setup();
+ admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInMessages", "5");
+ admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInBytes", "50");
final String topicName = "persistent://" + myNamespace + "/test-" + UUID.randomUUID();
pulsarClient.newProducer().topic(topicName).create().close();
@@ -2817,6 +2835,10 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception {
publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter();
Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 5);
Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 50L);
+
+ // restore defaults
+ admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInMessages", "0");
+ admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInBytes", "0");
}
@Test(timeOut = 20000)
@@ -3050,9 +3072,7 @@ public void testMaxSubscriptionsPerTopicWithExistingSubs() throws Exception {
@Test
public void testMaxUnackedMessagesOnSubscriptionPriority() throws Exception {
- cleanup();
- conf.setMaxUnackedMessagesPerSubscription(30);
- setup();
+ restartBroker(conf -> conf.setMaxUnackedMessagesPerSubscription(30));
final String topic = "persistent://" + myNamespace + "/test-" + UUID.randomUUID();
// init cache
@Cleanup
@@ -3115,6 +3135,9 @@ public void testMaxUnackedMessagesOnSubscriptionPriority() throws Exception {
&& admin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic) == null);
messages = getMsgReceived(consumer1, Integer.MAX_VALUE);
assertEquals(messages.size(), defaultMaxUnackedMsgOnBroker);
+
+ // restore default config
+ restartBroker(conf -> conf.setMaxUnackedMessagesPerSubscription(4 * 50000));
}
private void produceMsg(Producer producer, int msgNum) throws Exception{
@@ -3299,14 +3322,16 @@ public void testGetReplicatorRateApplied() throws Exception {
@Test(timeOut = 30000)
public void testAutoCreationDisabled() throws Exception {
- cleanup();
- conf.setAllowAutoTopicCreation(false);
- setup();
+ admin.brokers().updateDynamicConfiguration("allowAutoTopicCreation", "false");
+
final String topic = testTopic + UUID.randomUUID();
admin.topics().createPartitionedTopic(topic, 3);
pulsarClient.newProducer().topic(topic).create().close();
//should not fail
assertNull(admin.topicPolicies().getMessageTTL(topic));
+
+ // restore default
+ admin.brokers().updateDynamicConfiguration("allowAutoTopicCreation", "true");
}
@SuppressWarnings("deprecation")
@@ -3431,6 +3456,12 @@ public void testSubscriptionTypesEnabled() throws Exception {
pulsarClient.newConsumer().topic(topic)
.subscriptionType(SubscriptionType.Shared).subscriptionName("test")
.subscribe().close();
+
+ // restore dynamic broker config and conf object
+ pulsar.getConfiguration().setSubscriptionTypesEnabled(
+ Set.of("Exclusive", "Shared", "Failover", "Key_Shared"));
+ admin.brokers().updateDynamicConfiguration("subscriptionTypesEnabled",
+ "Exclusive,Shared,Failover,Key_Shared");
}
@Test(timeOut = 20000)
@@ -3765,7 +3796,8 @@ public void testPolicyIsDeleteTogetherAutomatically() throws Exception {
}
@Test
- public void testDoNotCreateSystemTopicForHeartbeatNamespace() {
+ public void testDoNotCreateSystemTopicForHeartbeatNamespace() throws Exception {
+ initEventsTopicAndPartitions();
assertTrue(pulsar.getBrokerService().getTopics().size() > 0);
pulsar.getBrokerService().getTopics().forEach((k, v) -> {
TopicName topicName = TopicName.get(k);
@@ -4121,6 +4153,7 @@ public void testGlobalTopicPolicies() throws Exception {
@Test
public void testMaxMessageSizeWithChunking() throws Exception {
+ final var maxMessageSize = this.conf.getMaxMessageSize();
this.conf.setMaxMessageSize(1000);
@Cleanup
@@ -4149,6 +4182,7 @@ public void testMaxMessageSizeWithChunking() throws Exception {
// chunk message send success
producer.send(new byte[2000]);
+ this.conf.setMaxMessageSize(maxMessageSize);
}
@Test(timeOut = 30000)
@@ -4202,6 +4236,7 @@ public void testGetTopicPoliciesWhenDeleteTopicPolicy() throws Exception {
@Test
public void testProduceChangesWithEncryptionRequired() throws Exception {
+ initEventsTopicAndPartitions();
final String beforeLac = admin.topics().getInternalStats(topicPolicyEventsTopic).lastConfirmedEntry;
admin.namespaces().setEncryptionRequiredStatus(myNamespace, true);
// just an update to trigger writes on __change_events
@@ -4657,4 +4692,9 @@ public void testGetAppliedOffloadPoliciesWithLegacyNamespacePolicies() throws Ex
assertEquals(offloadPolicies.getManagedLedgerOffloadThresholdInBytes(), (Long) (1024 * 1024 * 10L),
"Should inherit offload threshold from legacy namespace policy");
}
+
+ private void initEventsTopicAndPartitions() throws Exception {
+ Producer> producer = pulsarClient.newProducer().topic(testTopic).create();
+ producer.close();
+ }
}
From 7d10d6621699bc02d519cb6d9d08ddb529fc03b5 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Thu, 7 May 2026 11:24:46 +0800
Subject: [PATCH 12/18] reuse TopicPoliciesTest for
MetadataStoreTopicPoliciesService
---
.../LegacyAwareTopicPoliciesService.java | 6 ++
.../MetadataStoreTopicPoliciesService.java | 10 +++
.../admin/MetadataStoreTopicPoliciesTest.java | 72 +++++++++++++++++++
.../broker/admin/TopicPoliciesTest.java | 35 +++++----
.../broker/service/TopicPolicyTestUtils.java | 8 +++
5 files changed, 118 insertions(+), 13 deletions(-)
create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MetadataStoreTopicPoliciesTest.java
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index 6b472184f354d..b7f47d2ded65b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.service;
+import com.google.common.annotations.VisibleForTesting;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;
@@ -108,6 +109,11 @@ private CompletableFuture resolveService(NamespaceName nam
return isLegacy(namespace).thenApply(isLegacy -> isLegacy ? systemTopicService : configuredService);
}
+ @VisibleForTesting
+ public CompletableFuture resolveServiceForTesting(NamespaceName namespace) {
+ return resolveService(namespace);
+ }
+
private CompletableFuture isLegacy(NamespaceName namespace) {
return NamespaceEventsSystemTopicFactory.checkSystemTopicExists(namespace, EventType.TOPIC_POLICY, pulsar);
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index d6b6a94fef59b..97b9f073d236b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -256,6 +256,16 @@ private static TopicPolicies cloneWithScope(TopicPolicies policies, boolean isGl
return cloned;
}
+ @VisibleForTesting
+ public CompletableFuture> getTopicPoliciesDirectFromStore(TopicName topicName,
+ boolean isGlobal) {
+ TopicName partitionedTopicName = normalizeTopicName(topicName);
+ String path = pathFor(partitionedTopicName, isGlobal);
+ MetadataCache c = cache(isGlobal);
+ c.invalidate(path);
+ return c.get(path).thenApply(opt -> opt.map(p -> cloneWithScope(p, isGlobal)));
+ }
+
@VisibleForTesting
static String pathFor(TopicName topicName, boolean isGlobalPolicy) {
TopicName partitionedTopicName = normalizeTopicName(topicName);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MetadataStoreTopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MetadataStoreTopicPoliciesTest.java
new file mode 100644
index 0000000000000..e7fefa164973a
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MetadataStoreTopicPoliciesTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pulsar.broker.admin;
+
+import org.apache.pulsar.broker.service.MetadataStoreTopicPoliciesService;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-admin")
+public class MetadataStoreTopicPoliciesTest extends TopicPoliciesTest {
+
+ @BeforeClass(alwaysRun = true)
+ @Override
+ protected void setup() throws Exception {
+ conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
+ super.setup();
+ }
+
+ @Override
+ protected void clearTopicPoliciesCache() {
+ }
+
+ @Test(enabled = false)
+ @Override
+ public void testTopicPolicyInitialValueWithNamespaceAlreadyLoaded() throws Exception {
+ // This test is specific to SystemTopicBasedTopicPoliciesService (uses getPoliciesCacheInit).
+ // Not applicable to MetadataStoreTopicPoliciesService.
+ }
+
+ @Test(enabled = false)
+ @Override
+ public void testSystemTopicShouldBeCompacted() throws Exception {
+ // Relies on __change_events system topic, which does not exist with MetadataStoreTopicPoliciesService.
+ }
+
+ @Test(enabled = false)
+ @Override
+ public void testPoliciesCanBeDeletedWithTopic() throws Exception {
+ // Directly accesses __change_events PersistentTopic for compaction.
+ // Not applicable to MetadataStoreTopicPoliciesService.
+ }
+
+ @Test(enabled = false)
+ @Override
+ public void testProduceChangesWithEncryptionRequired() throws Exception {
+ // Checks __change_events LAC, which does not exist with MetadataStoreTopicPoliciesService.
+ }
+
+ @Test(enabled = false)
+ @Override
+ public void testTopicPoliciesAfterCompaction(String reloadPolicyType) throws Exception {
+ // The "Recreate_Service" variant creates a new SystemTopicBasedTopicPoliciesService,
+ // which is not applicable to MetadataStoreTopicPoliciesService.
+ }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
index 92d2e4561ae4b..02117ebd008a6 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
@@ -180,6 +180,10 @@ void setupTestTopic() throws Exception {
admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Set.of("test"));
admin.namespaces().createNamespace(myNamespaceV1);
admin.topics().createPartitionedTopic(testTopic, testTopicPartitions);
+ // Acquire namespace bundle ownership so tests that call getOrCreateTopic() directly succeed.
+ // Without this, services that don't create a __change_events reader (e.g. MetadataStoreTopicPoliciesService)
+ // leave the bundle unowned after namespace recreation and the first broker-side topic load fails.
+ admin.lookups().lookupTopic(testTopic + "-partition-0");
}
@AfterMethod
@@ -539,8 +543,8 @@ public Object[][] clientRequestType() {
@Test(dataProvider = "clientRequestType")
public void testPriorityOfGlobalPolicies(String clientRequestType) throws Exception {
- final SystemTopicBasedTopicPoliciesService topicPoliciesService =
- (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService();
+ final TopicPoliciesService topicPoliciesService =
+ pulsar.getTopicPoliciesService();
final JerseyClient httpClient = JerseyClientBuilder.createClient();
// create topic and load it up.
final String namespace = myNamespace;
@@ -620,8 +624,8 @@ public void testPriorityOfGlobalPolicies(String clientRequestType) throws Except
@Test(dataProvider = "clientRequestType")
public void testPriorityOfGlobalPolicies2(String clientRequestType) throws Exception {
- final SystemTopicBasedTopicPoliciesService topicPoliciesService =
- (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService();
+ final TopicPoliciesService topicPoliciesService =
+ pulsar.getTopicPoliciesService();
final JerseyClient httpClient = JerseyClientBuilder.createClient();
// create topic and load it up.
final String namespace = myNamespace;
@@ -707,8 +711,8 @@ public void testGlobalPolicyStillAffectsAfterUnloading() throws Exception {
final TopicName topicName = TopicName.get(topic);
admin.topics().createNonPartitionedTopic(topic);
pulsarClient.newProducer().topic(topic).create().close();
- final SystemTopicBasedTopicPoliciesService topicPoliciesService =
- (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService();
+ final TopicPoliciesService topicPoliciesService =
+ pulsar.getTopicPoliciesService();
// Set non-global policy of the limitation of max consumers.
// Set global policy of the limitation of max producers.
@@ -749,8 +753,8 @@ public void testRetentionGlobalPolicyAffects() throws Exception {
final TopicName topicName = TopicName.get(topic);
admin.topics().createNonPartitionedTopic(topic);
pulsarClient.newProducer().topic(topic).create().close();
- final SystemTopicBasedTopicPoliciesService topicPoliciesService =
- (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService();
+ final TopicPoliciesService topicPoliciesService =
+ pulsar.getTopicPoliciesService();
// Set non-global policy of the limitation of max consumers.
// Set global policy of the persistence policies.
@@ -3858,8 +3862,13 @@ public void testLoopCreateAndDeleteTopicPolicies() throws Exception {
}
private void triggerAndWaitNewTopicCompaction(String topicName) throws Exception {
- PersistentTopic tp =
- (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get();
+ Optional topicOpt =
+ pulsar.getBrokerService().getTopic(topicName, false).join();
+ if (topicOpt.isEmpty()) {
+ // Topic doesn't exist (e.g., when not using system-topic-based policies service), nothing to compact.
+ return;
+ }
+ PersistentTopic tp = (PersistentTopic) topicOpt.get();
// Wait for the old task finish.
Awaitility.await().untilAsserted(() -> {
CompletableFuture compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
@@ -3878,7 +3887,7 @@ private void triggerAndWaitNewTopicCompaction(String topicName) throws Exception
* It is not a thread safety method, something will go to a wrong pointer if there is a task is trying to load a
* topic policies.
*/
- private void clearTopicPoliciesCache() {
+ protected void clearTopicPoliciesCache() {
TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService();
if (topicPoliciesService instanceof TopicPoliciesService.TopicPoliciesServiceDisabled) {
return;
@@ -4108,8 +4117,8 @@ public void testGlobalTopicPolicies() throws Exception {
.isNull());
admin.topicPolicies(true).setRetention(topic, new RetentionPolicies(1,
2));
- SystemTopicBasedTopicPoliciesService topicPoliciesService =
- (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService();
+ TopicPoliciesService topicPoliciesService =
+ pulsar.getTopicPoliciesService();
// check global topic policies can be added correctly.
Awaitility.await().untilAsserted(() -> assertNotNull(
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java
index 6b9735d59b21a..e6ccc5fcf6e7e 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java
@@ -72,6 +72,14 @@ public static TopicPolicies getGlobalTopicPolicies(TopicPoliciesService topicPol
public static Optional getTopicPoliciesBypassCache(TopicPoliciesService topicPoliciesService,
TopicName topicName, boolean isGlobal)
throws Exception {
+ if (topicPoliciesService instanceof LegacyAwareTopicPoliciesService legacyService) {
+ TopicPoliciesService resolved =
+ legacyService.resolveServiceForTesting(topicName.getNamespaceObject()).get();
+ return getTopicPoliciesBypassCache(resolved, topicName, isGlobal);
+ }
+ if (topicPoliciesService instanceof MetadataStoreTopicPoliciesService metadataStoreService) {
+ return metadataStoreService.getTopicPoliciesDirectFromStore(topicName, isGlobal).get();
+ }
@Cleanup final var reader = ((SystemTopicBasedTopicPoliciesService) topicPoliciesService)
.getNamespaceEventsSystemTopicFactory()
.createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject())
From 055371f6de6f57f9c4d288213e005755e5c77aaf Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Thu, 7 May 2026 11:37:18 +0800
Subject: [PATCH 13/18] chore: simplify tests
---
.../LegacyAwareTopicPoliciesServiceTest.java | 58 ++++-
...MetadataStoreTopicPoliciesServiceTest.java | 204 ------------------
2 files changed, 55 insertions(+), 207 deletions(-)
delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index 539e215479abb..8951aaa567dc2 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -22,21 +22,38 @@
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.awaitility.Awaitility;
import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@Test(groups = "broker")
-public class LegacyAwareTopicPoliciesServiceTest extends MetadataStoreTopicPoliciesServiceTest{
+public class LegacyAwareTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
+ @BeforeClass
@Override
- protected boolean isLegacyTopicPoliciesService() {
- return true;
+ protected void setup() throws Exception {
+ conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
+ super.internalSetup();
+ super.setupDefaultTenantAndNamespace();
+ assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
+ }
+
+ @AfterClass
+ @Override
+ protected void cleanup() throws Exception {
+ super.internalCleanup();
}
@Test
@@ -180,4 +197,39 @@ private void restartWithLegacyAwareMetadataStoreService() throws Exception {
private PersistentTopic getPersistentTopic(String topic) throws Exception {
return (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElseThrow();
}
+
+ @Test
+ public void testListenerTriggered() throws Exception {
+ final var topic = TopicName.get("test-global-policies-not-triggered").toString();
+ final var topicName = TopicName.get(topic);
+ admin.topics().createNonPartitionedTopic(topic);
+
+ final var compactionThreshold = new AtomicLong(0);
+ pulsar.getTopicPoliciesService().registerListener(topicName, policies ->
+ Optional.ofNullable(policies).map(TopicPolicies::getCompactionThreshold).ifPresentOrElse(
+ compactionThreshold::set, () -> compactionThreshold.set(-1)));
+
+ // Verify Created events are handled
+ admin.topicPolicies(false).setCompactionThreshold(topic, 100);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 100));
+
+ admin.topicPolicies(true).setCompactionThreshold(topic, 200);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 200));
+
+ // Verify Modified events are handled
+ admin.topicPolicies(false).setCompactionThreshold(topic, 300);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 300));
+
+ admin.topicPolicies(true).setCompactionThreshold(topic, 400);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), 400));
+
+ // Verify Deleted events are handled
+ admin.topicPolicies(false).deleteTopicPolicies(topic);
+ Awaitility.await().atMost(Duration.ofSeconds(1))
+ .untilAsserted(() -> assertEquals(compactionThreshold.get(), -1));
+ }
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
deleted file mode 100644
index c2719ab7eb324..0000000000000
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesServiceTest.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pulsar.broker.service;
-
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNull;
-import static org.testng.Assert.assertTrue;
-import java.time.Duration;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
-import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.broker.testcontext.PulsarTestContext;
-import org.apache.pulsar.client.api.Schema;
-import org.apache.pulsar.common.naming.NamespaceBundleFactory;
-import org.apache.pulsar.common.naming.TopicName;
-import org.apache.pulsar.common.policies.data.TopicPolicies;
-import org.apache.pulsar.utils.TestLogAppender;
-import org.awaitility.Awaitility;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-@Test(groups = "broker")
-public class MetadataStoreTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
-
- protected boolean isLegacyTopicPoliciesService() {
- return false;
- }
-
- @Override
- protected void configureMetadataStores(PulsarTestContext.Builder builder) {
- builder.withMockZookeeper();
- }
-
- @BeforeClass
- @Override
- protected void setup() throws Exception {
- conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
- conf.setSystemTopicEnabled(isLegacyTopicPoliciesService());
- super.internalSetup();
- super.setupDefaultTenantAndNamespace();
- if (isLegacyTopicPoliciesService()) {
- assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
- } else {
- assertTrue(pulsar.getTopicPoliciesService() instanceof MetadataStoreTopicPoliciesService);
- }
- }
-
- @AfterClass
- @Override
- protected void cleanup() throws Exception {
- super.internalCleanup();
- }
-
- @Test
- public void testTopicPoliciesAdminPersistsAndUpdatesLoadedTopic() throws Exception {
- final var topic = TopicName.get("test-metadata-store-topic-policies").toString();
- admin.topics().createNonPartitionedTopic(topic);
-
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create()) {
- producer.send("warmup");
- }
- final var persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get()
- .orElseThrow();
-
- assertNull(admin.topicPolicies().getCompactionThreshold(topic));
- Assert.assertNotEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
- Long.valueOf(1000));
-
- admin.topicPolicies().setCompactionThreshold(topic, 1000);
- Awaitility.await().untilAsserted(() -> {
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic), 1000L);
- assertEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(), 1000L);
- });
-
- restartBroker();
-
- Awaitility.await().untilAsserted(() ->
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000)));
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create()) {
- producer.send("after-restart");
- }
- final var reloadedTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get()
- .orElseThrow();
- Awaitility.await().untilAsserted(() ->
- assertEquals(reloadedTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
- Long.valueOf(1000)));
- }
-
- @Test
- public void testTopicPoliciesDeletedWithTopic() throws Exception {
- final var topic = TopicName.get("test-metadata-store-topic-policies-delete").toString();
- admin.topics().createNonPartitionedTopic(topic);
-
- assertNull(admin.topicPolicies().getCompactionThreshold(topic));
- admin.topicPolicies().setCompactionThreshold(topic, 1000);
- Awaitility.await().untilAsserted(() ->
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000)));
-
- admin.topics().delete(topic);
- admin.topics().createNonPartitionedTopic(topic);
- Awaitility.await().untilAsserted(() -> assertNull(admin.topicPolicies().getCompactionThreshold(topic)));
- }
-
- // Verify the topic policies metadata path does not have conflicts with "/admin/local-policies", which is registered
- // with a listener from `NamespaceBundleFactory`.
- @Test
- public void testLocalTopicPoliciesPathDoesNotTriggerNamespaceBundleFactoryError() throws Exception {
- final var topic = TopicName.get("test-metadata-store-topic-policies-log-regression").toString();
- admin.topics().createNonPartitionedTopic(topic);
-
- try (var appender = TestLogAppender.create(NamespaceBundleFactory.class)) {
- appender.clearEvents();
-
- admin.topicPolicies().setCompactionThreshold(topic, 1000);
- Awaitility.await().untilAsserted(() ->
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000)));
-
- Awaitility.await().during(1, TimeUnit.SECONDS).atMost(2, TimeUnit.SECONDS).until(() ->
- appender.getEvents().stream().noneMatch(event ->
- event.getMessage().getFormattedMessage()
- .contains("Failed to update the policy change for path")));
- }
- }
-
- @Test
- public void testShadowReplicator() throws Exception {
- final var sourceTopic = TopicName.get("test-metadata-shadow-replicator").toString();
- final var shadowTopic = sourceTopic + "-shadow";
-
- admin.topics().createNonPartitionedTopic(sourceTopic);
- admin.topics().createShadowTopic(shadowTopic, sourceTopic);
- admin.topics().setShadowTopics(sourceTopic, List.of(shadowTopic));
-
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create();
- var consumer = pulsarClient.newConsumer(Schema.STRING).topic(shadowTopic)
- .subscriptionName("sub").subscribe()) {
- producer.send("msg");
- final var msg = consumer.receive(5, TimeUnit.SECONDS);
- Assert.assertNotNull(msg);
- Assert.assertEquals(msg.getValue(), "msg");
- }
-
- final var persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(sourceTopic).get()
- .orElseThrow();
- Awaitility.await().untilAsserted(() ->
- Assert.assertEquals(persistentTopic.getShadowReplicators().size(), 1));
- }
-
- @Test
- public void testListenerTriggered() throws Exception {
- final var topic = TopicName.get("test-global-policies-not-triggered").toString();
- final var topicName = TopicName.get(topic);
- admin.topics().createNonPartitionedTopic(topic);
-
- final var compactionThreshold = new AtomicLong(0);
- pulsar.getTopicPoliciesService().registerListener(topicName, policies ->
- Optional.ofNullable(policies).map(TopicPolicies::getCompactionThreshold).ifPresentOrElse(
- compactionThreshold::set, () -> compactionThreshold.set(-1)));
-
- // Verify Created events are handled
- admin.topicPolicies(false).setCompactionThreshold(topic, 100);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 100));
-
- admin.topicPolicies(true).setCompactionThreshold(topic, 200);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 200));
-
- // Verify Modified events are handled
- admin.topicPolicies(false).setCompactionThreshold(topic, 300);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 300));
-
- admin.topicPolicies(true).setCompactionThreshold(topic, 400);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 400));
-
- // Verify Deleted events are handled
- admin.topicPolicies(false).deleteTopicPolicies(topic);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), -1));
- }
-}
From a10dd0bd55aff597d4a065e9324671f53653cd95 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Thu, 7 May 2026 12:31:25 +0800
Subject: [PATCH 14/18] chore: improve tests for legacy aware
---
.../pulsar/broker/ServiceConfiguration.java | 12 +-
.../MetadataStoreTopicPoliciesService.java | 4 +-
.../LegacyAwareTopicPoliciesServiceTest.java | 219 ++++++------------
.../service/TopicPoliciesServiceInitTest.java | 101 --------
4 files changed, 81 insertions(+), 255 deletions(-)
delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index b22f9a4bef827..632df19a302a9 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -1750,8 +1750,16 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece
@FieldContext(
category = CATEGORY_SERVER,
- doc = "The class name of the topic policies service. The default config only takes affect when the "
- + "systemTopicEnable config is true"
+ doc = """
+ The class name of the topic policies service. There are 2 built-in implementations:
+ 1. "org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService" (default)
+ It stores a topic's policies in the `__change_events` topic. If `systemTopicEnabled` is false,
+ the topic policies will just be disabled
+ 2. "org.apache.pulsar.broker.service.MetadataStoreTopicPoliciesService"
+ It stores a topic's policies in the metadata store. If `systemTopicEnabled` is true and the
+ topic's namespace has a `__change_events` topic, the policies will still be stored in the
+ `__change_events` topic for backward compatibility.
+ """
)
private String topicPoliciesServiceClassName =
"org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService";
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index 97b9f073d236b..b76de3c371eec 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -48,8 +48,8 @@
@CustomLog
public class MetadataStoreTopicPoliciesService implements TopicPoliciesService {
- private static final String GLOBAL_POLICIES_ROOT = "/admin/topic-policies/global";
- private static final String LOCAL_POLICIES_ROOT = "/admin/topic-policies/local";
+ public static final String GLOBAL_POLICIES_ROOT = "/admin/topic-policies/global";
+ public static final String LOCAL_POLICIES_ROOT = "/admin/topic-policies/local";
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Map> listeners = new ConcurrentHashMap<>();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index 8951aaa567dc2..35c772036c10f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -26,28 +26,27 @@
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
-import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory;
-import org.apache.pulsar.client.api.Schema;
-import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.awaitility.Awaitility;
-import org.testng.Assert;
+import org.awaitility.core.ThrowingRunnable;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
+/**
+ * Test order: testUpgrade() -> other tests (with MetadataStoreTopicPoliciesService configured) -> testDowngrade().
+ */
@Test(groups = "broker")
public class LegacyAwareTopicPoliciesServiceTest extends MockedPulsarServiceBaseTest {
+ private static final String metaNamespace = "public/meta-ns";
+
@BeforeClass
@Override
protected void setup() throws Exception {
- conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
super.internalSetup();
super.setupDefaultTenantAndNamespace();
- assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
}
@AfterClass
@@ -56,180 +55,100 @@ protected void cleanup() throws Exception {
super.internalCleanup();
}
- @Test
- public void testLegacyNamespaceKeepsSystemTopicBackendAfterRestart() throws Exception {
- restartWithSystemTopicPoliciesService();
-
- final var namespace1 = "public/legacy-aware-ns1";
- final var topic1 = "persistent://" + namespace1 + "/topic";
- final var eventTopic1 = NamespaceEventsSystemTopicFactory
- .getEventsTopicName(NamespaceName.get(namespace1))
- .toString();
- admin.namespaces().createNamespace(namespace1);
- admin.topics().createNonPartitionedTopic(topic1);
-
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic1).create()) {
- producer.send("warmup");
- }
- final var namespace1TopicBeforeRestart = getPersistentTopic(topic1);
-
- assertNull(admin.topicPolicies().getCompactionThreshold(topic1));
-
- admin.topicPolicies().setCompactionThreshold(topic1, 1000);
- Awaitility.await().untilAsserted(() -> {
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), Long.valueOf(1000));
- assertEquals(namespace1TopicBeforeRestart.getHierarchyTopicPolicies().getCompactionThreshold().get(),
- Long.valueOf(1000));
- assertTrue(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic1))
- .join());
- });
+ @Test(priority = -1)
+ public void testUpgrade() throws Exception {
+ final var topic = "test-upgrade";
+ admin.topics().createNonPartitionedTopic(topic);
+ admin.topicPolicies().setCompactionThreshold(topic, 100);
+ waitUntilAssert(() -> assertEquals(admin.topicPolicies().getCompactionThreshold(topic), 100));
- restartBroker(configuration ->
- configuration.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName()));
- assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
-
- Awaitility.await().untilAsserted(() ->
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), Long.valueOf(1000)));
-
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic1).create()) {
- producer.send("after-restart");
- }
- final var namespace1TopicAfterRestart = getPersistentTopic(topic1);
- final var namespace1LacBeforeUpdate = admin.topics().getInternalStats(eventTopic1).lastConfirmedEntry;
-
- admin.topicPolicies().setCompactionThreshold(topic1, 2000);
- Awaitility.await().untilAsserted(() -> {
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), Long.valueOf(2000));
- assertEquals(namespace1TopicAfterRestart.getHierarchyTopicPolicies().getCompactionThreshold().get(),
- Long.valueOf(2000));
- Assert.assertNotEquals(admin.topics().getInternalStats(eventTopic1).lastConfirmedEntry,
- namespace1LacBeforeUpdate);
+ restartBroker(conf -> {
+ conf.setSystemTopicEnabled(false);
+ conf.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
});
- assertFalse(pulsar.getLocalMetadataStore()
- .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic1), false))
- .join());
-
- final var namespace2 = "public/legacy-aware-ns2";
- final var topic2 = "persistent://" + namespace2 + "/topic";
- final var eventTopic2 = NamespaceEventsSystemTopicFactory
- .getEventsTopicName(NamespaceName.get(namespace2))
- .toString();
- admin.namespaces().createNamespace(namespace2);
- admin.topics().createNonPartitionedTopic(topic2);
-
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic2).create()) {
- producer.send("warmup");
- }
- final var namespace2Topic = getPersistentTopic(topic2);
+ // The policies will be lost because when system topic is disabled, it will not try to read policies from the
+ // __change_events topic
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic));
- assertNull(admin.topicPolicies().getCompactionThreshold(topic2));
- assertFalse(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic2))
- .join());
-
- admin.topicPolicies().setCompactionThreshold(topic2, 3000);
- Awaitility.await().untilAsserted(() -> {
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic2), Long.valueOf(3000));
- assertEquals(namespace2Topic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
- Long.valueOf(3000));
- assertFalse(pulsar.getPulsarResources().getTopicResources()
- .persistentTopicExists(TopicName.get(eventTopic2)).join());
- assertTrue(pulsar.getLocalMetadataStore()
- .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic2), false))
- .join());
- });
- }
+ restartBroker(conf -> conf.setSystemTopicEnabled(true));
+ // The default namespace still read policies from the __change_events topic if it exists
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic), 100);
+ assertFalse(pulsar.getLocalMetadataStore().exists(MetadataStoreTopicPoliciesService.LOCAL_POLICIES_ROOT).get());
- @Test
- public void testOwnedBundleCanSwitchToLegacyBackendAfterNamespaceBecomesLegacy() throws Exception {
- restartWithLegacyAwareMetadataStoreService();
-
- final var namespace = "public/legacy-aware-flip";
- final var topic = "persistent://" + namespace + "/topic";
- final var eventTopic = NamespaceEventsSystemTopicFactory
- .getEventsTopicName(NamespaceName.get(namespace))
- .toString();
- admin.namespaces().createNamespace(namespace);
- admin.topics().createNonPartitionedTopic(topic);
+ // The global policies are still stored in the __change_events topic
+ admin.topicPolicies(true).setCompactionThreshold(topic, 200);
+ waitUntilAssert(() -> assertEquals(admin.topicPolicies(true).getCompactionThreshold(topic), 200));
+ assertFalse(pulsar.getConfigurationMetadataStore()
+ .exists(MetadataStoreTopicPoliciesService.GLOBAL_POLICIES_ROOT).get());
- try (var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create()) {
- producer.send("warmup");
- }
- final var persistentTopic = getPersistentTopic(topic);
+ admin.topicPolicies().deleteTopicPolicies(topic);
+ waitUntilAssert(() -> assertNull(admin.topicPolicies().getCompactionThreshold(topic)));
- assertNull(admin.topicPolicies().getCompactionThreshold(topic));
- assertFalse(pulsar.getPulsarResources().getTopicResources().persistentTopicExists(TopicName.get(eventTopic))
- .join());
- assertFalse(pulsar.getLocalMetadataStore()
- .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic), false))
- .join());
-
- admin.topics().createNonPartitionedTopic(eventTopic);
- Awaitility.await().untilAsserted(() -> assertTrue(pulsar.getPulsarResources().getTopicResources()
- .persistentTopicExists(TopicName.get(eventTopic)).join()));
-
- final var eventTopicLastConfirmedEntryBeforeUpdate = admin.topics().getInternalStats(eventTopic)
- .lastConfirmedEntry;
- admin.topicPolicies().setCompactionThreshold(topic, 1000);
- Awaitility.await().untilAsserted(() -> {
- assertEquals(admin.topicPolicies().getCompactionThreshold(topic), Long.valueOf(1000));
- assertEquals(persistentTopic.getHierarchyTopicPolicies().getCompactionThreshold().get(),
- Long.valueOf(1000));
- Assert.assertNotEquals(admin.topics().getInternalStats(eventTopic).lastConfirmedEntry,
- eventTopicLastConfirmedEntryBeforeUpdate);
- assertFalse(pulsar.getLocalMetadataStore()
- .exists(MetadataStoreTopicPoliciesService.pathFor(TopicName.get(topic), false))
- .join());
- });
+ admin.namespaces().createNamespace(metaNamespace);
}
- private void restartWithSystemTopicPoliciesService() throws Exception {
- restartBroker(configuration ->
- configuration.setTopicPoliciesServiceClassName(SystemTopicBasedTopicPoliciesService.class.getName()));
- assertTrue(pulsar.getTopicPoliciesService() instanceof SystemTopicBasedTopicPoliciesService);
- }
+ @Test(priority = 1)
+ public void testDowngrade() throws Exception {
+ final var topic1 = "downgrade"; // in default namespace
+ admin.topics().createNonPartitionedTopic(topic1);
+ admin.topicPolicies().setCompactionThreshold(topic1, 1);
+ waitUntilAssert(() -> assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), 1));
- private void restartWithLegacyAwareMetadataStoreService() throws Exception {
- restartBroker(configuration ->
- configuration.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName()));
- assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
- }
+ final var topic2 = metaNamespace + "/downgrade";
+ admin.topics().createNonPartitionedTopic(topic2);
+ admin.topicPolicies().setCompactionThreshold(topic2, 2);
+ waitUntilAssert(() -> assertEquals(admin.topicPolicies().getCompactionThreshold(topic2), 2));
- private PersistentTopic getPersistentTopic(String topic) throws Exception {
- return (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElseThrow();
+ restartBroker(conf ->
+ conf.setTopicPoliciesServiceClassName(SystemTopicBasedTopicPoliciesService.class.getName()));
+ assertEquals(admin.topicPolicies().getCompactionThreshold(topic1), 1);
+ // The policies will be lost because they are not stored in the __change_events topic
+ assertNull(admin.topicPolicies().getCompactionThreshold(topic2));
}
@Test
- public void testListenerTriggered() throws Exception {
- final var topic = TopicName.get("test-global-policies-not-triggered").toString();
- final var topicName = TopicName.get(topic);
+ public void testPoliciesStoredInMetadataStore() throws Exception {
+ final var topicName = TopicName.get(metaNamespace + "/test-policies-stored-in-metadata-store");
+ final var topic = topicName.toString();
admin.topics().createNonPartitionedTopic(topic);
final var compactionThreshold = new AtomicLong(0);
+ // Verify the exception thrown from one listener does not affect other listeners
+ pulsar.getTopicPoliciesService().registerListener(topicName, __ -> {
+ throw new RuntimeException("injected failure");
+ });
pulsar.getTopicPoliciesService().registerListener(topicName, policies ->
Optional.ofNullable(policies).map(TopicPolicies::getCompactionThreshold).ifPresentOrElse(
compactionThreshold::set, () -> compactionThreshold.set(-1)));
// Verify Created events are handled
admin.topicPolicies(false).setCompactionThreshold(topic, 100);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 100));
+ waitUntilAssert(() -> assertEquals(compactionThreshold.get(), 100));
+ final var localStore = pulsar.getLocalMetadataStore();
+ final var configurationStore = pulsar.getConfigurationMetadataStore();
+
+ assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, false)).get());
+ assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
admin.topicPolicies(true).setCompactionThreshold(topic, 200);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 200));
+ waitUntilAssert(() -> assertEquals(compactionThreshold.get(), 200));
+ assertTrue(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
// Verify Modified events are handled
admin.topicPolicies(false).setCompactionThreshold(topic, 300);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 300));
+ waitUntilAssert(() -> assertEquals(compactionThreshold.get(), 300));
admin.topicPolicies(true).setCompactionThreshold(topic, 400);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), 400));
+ waitUntilAssert(() -> assertEquals(compactionThreshold.get(), 400));
// Verify Deleted events are handled
admin.topicPolicies(false).deleteTopicPolicies(topic);
- Awaitility.await().atMost(Duration.ofSeconds(1))
- .untilAsserted(() -> assertEquals(compactionThreshold.get(), -1));
+ waitUntilAssert(() -> assertEquals(compactionThreshold.get(), -1));
+ assertFalse(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, false)).get());
+ assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ }
+
+ private static void waitUntilAssert(ThrowingRunnable assertion) {
+ Awaitility.await().atMost(Duration.ofSeconds(1)).untilAsserted(assertion);
}
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java
deleted file mode 100644
index 4349a1181dac8..0000000000000
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPoliciesServiceInitTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.pulsar.broker.service;
-
-import static org.testng.Assert.assertTrue;
-import static org.testng.Assert.fail;
-import java.util.Optional;
-import lombok.Cleanup;
-import org.apache.pulsar.broker.PulsarServerException;
-import org.apache.pulsar.broker.PulsarService;
-import org.apache.pulsar.broker.ServiceConfiguration;
-import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-@Test(groups = "broker")
-public class TopicPoliciesServiceInitTest {
-
- private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(1, 0, () -> 0);
- private ServiceConfiguration config;
-
- @BeforeClass
- public void setup() throws Exception {
- bk.start();
- }
-
- @AfterClass
- public void teardown() throws Exception {
- bk.stop();
- }
-
- @BeforeMethod
- public void initConfig() {
- config = new ServiceConfiguration();
- config.setClusterName("testCluster");
- config.setMetadataStoreUrl("zk:127.0.0.1:" + bk.getZookeeperPort());
- config.setWebServicePort(Optional.of(0));
- config.setBrokerServicePort(Optional.of(0));
- config.setAdvertisedAddress("127.0.0.1");
- config.setManagedLedgerDefaultWriteQuorum(1);
- config.setManagedLedgerDefaultAckQuorum(1);
- config.setManagedLedgerDefaultEnsembleSize(1);
- }
-
- @Test
- public void testSystemTopicPoliciesService() throws Exception {
- config.setTopicPoliciesServiceClassName(SystemTopicBasedTopicPoliciesService.class.getName());
- @Cleanup final var pulsar = new PulsarService(config);
- pulsar.start();
- assertTrue(pulsar.getTopicPoliciesService() instanceof SystemTopicBasedTopicPoliciesService);
- }
-
- @Test
- public void testLegacyAwareTopicPoliciesService() throws Exception {
- config.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
- @Cleanup final var pulsar = new PulsarService(config);
- pulsar.start();
- assertTrue(pulsar.getTopicPoliciesService() instanceof LegacyAwareTopicPoliciesService);
- }
-
- @Test
- public void testMetadataStoreTopicPoliciesService() throws Exception {
- config.setTopicPoliciesServiceClassName(MetadataStoreTopicPoliciesService.class.getName());
- // the topic policies service won't be aware of the legacy `__change_events` system topics
- config.setSystemTopicEnabled(false);
- @Cleanup final var pulsar = new PulsarService(config);
- pulsar.start();
- assertTrue(pulsar.getTopicPoliciesService() instanceof MetadataStoreTopicPoliciesService);
- }
-
- @Test
- public void testWrongInitialization() throws Exception {
- config.setTopicPoliciesServiceClassName(LegacyAwareTopicPoliciesService.class.getName());
- @Cleanup final var pulsar = new PulsarService(config);
- try {
- pulsar.start();
- fail();
- } catch (PulsarServerException e) {
- assertTrue(e.getCause().getCause() instanceof NoSuchMethodException);
- }
- }
-}
From 3e3ccb57ded022d0dca3566f7dae1d00e9f1f1cb Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Thu, 7 May 2026 13:03:45 +0800
Subject: [PATCH 15/18] fix: ignore the __change_events topic created later
---
.../LegacyAwareTopicPoliciesService.java | 37 ++++++++++++++-----
.../LegacyAwareTopicPoliciesServiceTest.java | 16 ++++++++
.../broker/service/TopicPolicyTestUtils.java | 3 +-
3 files changed, 44 insertions(+), 12 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index b7f47d2ded65b..67dfe6eb4ec59 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -18,9 +18,14 @@
*/
package org.apache.pulsar.broker.service;
+import com.github.benmanes.caffeine.cache.AsyncCacheLoader;
+import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
import com.google.common.annotations.VisibleForTesting;
+import java.time.Duration;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
import java.util.function.Consumer;
import lombok.CustomLog;
import org.apache.pulsar.broker.PulsarService;
@@ -29,6 +34,7 @@
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.TopicPolicies;
+import org.jspecify.annotations.NonNull;
/**
* Routes topic policy operations to the legacy system-topic backend when a namespace already has
@@ -37,6 +43,19 @@
@CustomLog
public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
+ // Generally, we only need to check if the __change_events topic exists once because the __change_events topic
+ // should only be created by broker before the upgrade, where `SystemTopicBasedTopicPoliciesService` is configured
+ // as the topic policies service.
+ private final AsyncLoadingCache isLegacyNamespace = Caffeine.newBuilder()
+ .expireAfterWrite(Duration.ofHours(1))
+ .buildAsync(new AsyncCacheLoader<>() {
+ @NonNull
+ @Override
+ public CompletableFuture extends Boolean> asyncLoad(NamespaceName key, Executor executor) {
+ return NamespaceEventsSystemTopicFactory.checkSystemTopicExists(key, EventType.TOPIC_POLICY,
+ pulsar);
+ }
+ });
private final PulsarService pulsar;
private final SystemTopicBasedTopicPoliciesService systemTopicService;
private final TopicPoliciesService configuredService;
@@ -47,6 +66,10 @@ public LegacyAwareTopicPoliciesService(PulsarService pulsar,
this.pulsar = pulsar;
this.systemTopicService = systemTopicService;
this.configuredService = configuredService;
+ if (configuredService instanceof SystemTopicBasedTopicPoliciesService) {
+ throw new IllegalArgumentException(
+ "configuredService should not be an instance of SystemTopicBasedTopicPoliciesService");
+ }
}
@Override
@@ -94,6 +117,7 @@ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName,
@Override
public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
+ // It's okay to register listeners for both because only one listener will receive the updates
boolean configuredRegistered = configuredService.registerListener(topicName, listener);
boolean systemTopicRegistered = systemTopicService.registerListener(topicName, listener);
return configuredRegistered || systemTopicRegistered;
@@ -105,16 +129,9 @@ public void unregisterListener(TopicName topicName, TopicPolicyListener listener
systemTopicService.unregisterListener(topicName, listener);
}
- private CompletableFuture resolveService(NamespaceName namespace) {
- return isLegacy(namespace).thenApply(isLegacy -> isLegacy ? systemTopicService : configuredService);
- }
-
@VisibleForTesting
- public CompletableFuture resolveServiceForTesting(NamespaceName namespace) {
- return resolveService(namespace);
- }
-
- private CompletableFuture isLegacy(NamespaceName namespace) {
- return NamespaceEventsSystemTopicFactory.checkSystemTopicExists(namespace, EventType.TOPIC_POLICY, pulsar);
+ CompletableFuture resolveService(NamespaceName namespace) {
+ return isLegacyNamespace.get(namespace)
+ .thenApply(isLegacy -> isLegacy ? systemTopicService : configuredService);
}
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index 35c772036c10f..9d6bfbaf21318 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -26,6 +26,7 @@
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.common.naming.SystemTopicNames;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.awaitility.Awaitility;
@@ -148,6 +149,21 @@ public void testPoliciesStoredInMetadataStore() throws Exception {
assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
}
+ @Test
+ public void testUserCreatedEventsTopicAreIgnored() throws Exception {
+ final var topic = TopicName.get(metaNamespace + "/" + System.currentTimeMillis()).toString();
+ admin.topics().createNonPartitionedTopic(topic);
+ admin.topicPolicies().setCompactionThreshold(topic, 1);
+ waitUntilAssert(() -> assertEquals(admin.topicPolicies().getCompactionThreshold(topic), 1));
+
+ final var eventsTopic = metaNamespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME;
+ admin.topics().createNonPartitionedTopic(eventsTopic);
+ // Even if the __change_events topic is created, since it has detected the namespace didn't have the events
+ // topic before, it will be ignored and the policies are still read from metadata store.
+ waitUntilAssert(() -> assertEquals(admin.topicPolicies().getCompactionThreshold(topic), 1));
+ admin.topics().delete(eventsTopic);
+ }
+
private static void waitUntilAssert(ThrowingRunnable assertion) {
Awaitility.await().atMost(Duration.ofSeconds(1)).untilAsserted(assertion);
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java
index e6ccc5fcf6e7e..7e9c697fb5d03 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java
@@ -73,8 +73,7 @@ public static Optional getTopicPoliciesBypassCache(TopicPoliciesS
TopicName topicName, boolean isGlobal)
throws Exception {
if (topicPoliciesService instanceof LegacyAwareTopicPoliciesService legacyService) {
- TopicPoliciesService resolved =
- legacyService.resolveServiceForTesting(topicName.getNamespaceObject()).get();
+ TopicPoliciesService resolved = legacyService.resolveService(topicName.getNamespaceObject()).get();
return getTopicPoliciesBypassCache(resolved, topicName, isGlobal);
}
if (topicPoliciesService instanceof MetadataStoreTopicPoliciesService metadataStoreService) {
From 6b43d767391b50940ff392a3c2a6c1b0a04c2bf4 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Thu, 7 May 2026 13:15:10 +0800
Subject: [PATCH 16/18] chore: remove unnecessary null checks
---
.../broker/service/MetadataStoreTopicPoliciesService.java | 8 --------
1 file changed, 8 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
index b76de3c371eec..10fec22292851 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/MetadataStoreTopicPoliciesService.java
@@ -18,7 +18,6 @@
*/
package org.apache.pulsar.broker.service;
-import static java.util.Objects.requireNonNull;
import com.google.common.annotations.VisibleForTesting;
import java.util.List;
import java.util.Map;
@@ -95,8 +94,6 @@ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName,
public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boolean isGlobalPolicy,
boolean skipUpdateWhenTopicPolicyDoesntExist,
Consumer policyUpdater) {
- requireNonNull(topicName);
- requireNonNull(policyUpdater);
TopicName partitionedTopicName = normalizeTopicName(topicName);
if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
return CompletableFuture.failedFuture(new BrokerServiceException.NotAllowedException(
@@ -126,7 +123,6 @@ public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, boo
@Override
public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) {
- requireNonNull(topicName);
TopicName partitionedTopicName = normalizeTopicName(topicName);
if (NamespaceService.isHeartbeatNamespace(partitionedTopicName.getNamespaceObject())) {
return CompletableFuture.completedFuture(Optional.empty());
@@ -141,8 +137,6 @@ public CompletableFuture> getTopicPoliciesAsync(TopicNam
@Override
public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
- requireNonNull(topicName);
- requireNonNull(listener);
listeners.compute(normalizeTopicName(topicName), (__, topicListeners) -> {
if (topicListeners == null) {
topicListeners = new CopyOnWriteArrayList<>();
@@ -155,8 +149,6 @@ public boolean registerListener(TopicName topicName, TopicPolicyListener listene
@Override
public void unregisterListener(TopicName topicName, TopicPolicyListener listener) {
- requireNonNull(topicName);
- requireNonNull(listener);
listeners.computeIfPresent(normalizeTopicName(topicName), (__, topicListeners) -> {
topicListeners.remove(listener);
return topicListeners.isEmpty() ? null : topicListeners;
From b70a56714861abe9f3f11c28595ac2636077c101 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Thu, 7 May 2026 14:55:42 +0800
Subject: [PATCH 17/18] address comments
---
.../LegacyAwareTopicPoliciesService.java | 30 +++++++++----------
.../broker/admin/TopicPoliciesTest.java | 27 ++++++++++-------
2 files changed, 32 insertions(+), 25 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index 67dfe6eb4ec59..e3abde0289922 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -43,27 +43,26 @@
@CustomLog
public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
- // Generally, we only need to check if the __change_events topic exists once because the __change_events topic
- // should only be created by broker before the upgrade, where `SystemTopicBasedTopicPoliciesService` is configured
- // as the topic policies service.
- private final AsyncLoadingCache isLegacyNamespace = Caffeine.newBuilder()
- .expireAfterWrite(Duration.ofHours(1))
- .buildAsync(new AsyncCacheLoader<>() {
- @NonNull
- @Override
- public CompletableFuture extends Boolean> asyncLoad(NamespaceName key, Executor executor) {
- return NamespaceEventsSystemTopicFactory.checkSystemTopicExists(key, EventType.TOPIC_POLICY,
- pulsar);
- }
- });
- private final PulsarService pulsar;
+ private final AsyncLoadingCache isLegacyNamespace;
private final SystemTopicBasedTopicPoliciesService systemTopicService;
private final TopicPoliciesService configuredService;
public LegacyAwareTopicPoliciesService(PulsarService pulsar,
SystemTopicBasedTopicPoliciesService systemTopicService,
TopicPoliciesService configuredService) {
- this.pulsar = pulsar;
+ // Generally, we only need to check if the __change_events topic exists once because the __change_events topic
+ // should only be created by broker before the upgrade, where `SystemTopicBasedTopicPoliciesService` is
+ // configured as the topic policies service.
+ this.isLegacyNamespace = Caffeine.newBuilder().expireAfterWrite(Duration.ofHours(1))
+ .buildAsync(new AsyncCacheLoader<>() {
+ @NonNull
+ @Override
+ public CompletableFuture extends Boolean> asyncLoad(NamespaceName key,
+ @NonNull Executor executor) {
+ return NamespaceEventsSystemTopicFactory.checkSystemTopicExists(key, EventType.TOPIC_POLICY,
+ pulsar);
+ }
+ });
this.systemTopicService = systemTopicService;
this.configuredService = configuredService;
if (configuredService instanceof SystemTopicBasedTopicPoliciesService) {
@@ -74,6 +73,7 @@ public LegacyAwareTopicPoliciesService(PulsarService pulsar,
@Override
public void start(PulsarService pulsarService) {
+ systemTopicService.start(pulsarService);
configuredService.start(pulsarService);
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
index 02117ebd008a6..5ba979534ef55 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
@@ -175,8 +175,16 @@ void setupTestTopic() throws Exception {
} catch (PulsarAdminException.NotFoundException e) {
// topic may already be deleted
}
- admin.namespaces().deleteNamespace(myNamespace, true);
- admin.namespaces().deleteNamespace(myNamespaceV1, true);
+ try {
+ admin.namespaces().deleteNamespace(myNamespace, true);
+ } catch (PulsarAdminException.NotFoundException e) {
+ // namespace may already be deleted
+ }
+ try {
+ admin.namespaces().deleteNamespace(myNamespaceV1, true);
+ } catch (PulsarAdminException.NotFoundException e) {
+ // namespace may already be deleted
+ }
admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Set.of("test"));
admin.namespaces().createNamespace(myNamespaceV1);
admin.topics().createPartitionedTopic(testTopic, testTopicPartitions);
@@ -186,8 +194,10 @@ void setupTestTopic() throws Exception {
admin.lookups().lookupTopic(testTopic + "-partition-0");
}
- @AfterMethod
- void afterMethodCleanup() {
+ @AfterMethod(alwaysRun = true)
+ void afterMethodCleanup() throws Exception{
+ admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInMessages", "0");
+ admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInBytes", "0");
clearTopicPoliciesCache();
}
@@ -2839,10 +2849,6 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception {
publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter();
Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 5);
Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 50L);
-
- // restore defaults
- admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInMessages", "0");
- admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInBytes", "0");
}
@Test(timeOut = 20000)
@@ -4703,7 +4709,8 @@ public void testGetAppliedOffloadPoliciesWithLegacyNamespacePolicies() throws Ex
}
private void initEventsTopicAndPartitions() throws Exception {
- Producer> producer = pulsarClient.newProducer().topic(testTopic).create();
- producer.close();
+ try (Producer> producer = pulsarClient.newProducer().topic(testTopic).create()) {
+ // No-op. Creating the producer initializes the events topic and partitions.
+ }
}
}
From 4c33be9cef12bec57f114070023981eee6341bd2 Mon Sep 17 00:00:00 2001
From: Yunze Xu
Date: Fri, 8 May 2026 16:01:34 +0800
Subject: [PATCH 18/18] fix: add registerListenerAsync and skip start system
topic policies
---
pip/pip-469.md | 29 +++++++++++-
.../pulsar/broker/service/AbstractTopic.java | 2 +-
.../LegacyAwareTopicPoliciesService.java | 19 +++++---
.../broker/service/TopicPoliciesService.java | 15 +++++--
.../service/persistent/PersistentTopic.java | 8 ++--
.../LegacyAwareTopicPoliciesServiceTest.java | 44 ++++++++++++++-----
...temTopicBasedTopicPoliciesServiceTest.java | 4 +-
7 files changed, 92 insertions(+), 29 deletions(-)
diff --git a/pip/pip-469.md b/pip/pip-469.md
index 69036381f0f01..c1976bd104374 100644
--- a/pip/pip-469.md
+++ b/pip/pip-469.md
@@ -105,8 +105,15 @@ Broker startup validates both backends:
- `SystemTopicBasedTopicPoliciesService` must be instantiable.
- The configured `topicPoliciesServiceClassName` must be instantiable.
-If either backend cannot be instantiated or started, broker startup fails. There is no per-request fallback from one
-backend to another.
+`LegacyAwareTopicPoliciesService#start` starts only the configured backend. It intentionally does not call
+`SystemTopicBasedTopicPoliciesService#start`, because that start path registers a namespace-bundle ownership listener
+whose only purpose is to eagerly create a reader on `/__change_events` when a namespace bundle is loaded.
+Under legacy-aware routing, that eager optimization would be counterproductive because it can create readers for
+namespaces that do not have topic policies in `__change_events`. For legacy namespaces, the system-topic reader and
+policy cache are initialized lazily by the routed system-topic backend operations.
+
+If either backend cannot be instantiated, or if the configured backend cannot be started, broker startup fails. There is
+no per-request fallback from one backend to another.
### Namespace-scoped service routing
@@ -118,6 +125,10 @@ backend to another.
the system-topic backend when the system topic exists.
- Routing the same operations to the configured backend when the system topic does not exist.
+Listener registration is routed through `TopicPoliciesService#registerListenerAsync`. This lets the wrapper resolve the
+namespace backend before registering the listener, and the listener is registered only on the selected backend instead
+of being registered on both backends.
+
The system-topic existence check can be cached per namespace in memory, but the routing rule is defined by actual topic
existence rather than by new namespace metadata.
@@ -159,6 +170,11 @@ managed-ledger metadata updates.
### Listener behavior
+`TopicPoliciesService` adds `registerListenerAsync(TopicName, TopicPolicyListener)` for listener registration. The
+existing synchronous `registerListener(TopicName, TopicPolicyListener)` method is retained as a deprecated compatibility
+hook for existing custom implementations, and the default async method delegates to it. Implementations that need async
+routing or initialization, such as `LegacyAwareTopicPoliciesService`, override `registerListenerAsync` directly.
+
The backend registers watchers on both metadata stores:
- A change on the local path re-reads the local node and notifies listeners with the latest local `TopicPolicies` or
@@ -173,6 +189,11 @@ append-only replay log; it relies on metadata-store notifications and read-after
### Public API
+The `TopicPoliciesService` extension point gains a default
+`CompletableFuture registerListenerAsync(TopicName, TopicPolicyListener)` method. Existing implementations
+remain compatible because `registerListener(TopicName, TopicPolicyListener)` is retained and used by the default async
+implementation.
+
No new namespace policy field is introduced.
No new namespace admin REST endpoint or Java admin client method is introduced.
@@ -221,6 +242,10 @@ This upgrade rule is intentionally conservative:
This means some namespaces with an empty but already-created `__change_events` topic may continue using the
system-topic backend. That is acceptable because it avoids missing legacy state.
+Existing custom `TopicPoliciesService` implementations that only implement the synchronous `registerListener` method
+continue to work through the default `registerListenerAsync` bridge. Implementations can override
+`registerListenerAsync` when registration itself needs asynchronous backend resolution or initialization.
+
## Downgrade / Rollback
Rolling back to a broker version that does not understand legacy-aware routing returns topic-policies backend
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
index 4d29c751f03a9..3ad9eb438104a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
@@ -559,7 +559,7 @@ protected boolean isProducersExceeded(boolean isRemote) {
protected void registerTopicPolicyListener() {
brokerService.getPulsar().getTopicPoliciesService()
- .registerListener(TopicName.getPartitionedTopicName(topic), this);
+ .registerListenerAsync(TopicName.getPartitionedTopicName(topic), this);
}
protected void unregisterTopicPolicyListener() {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
index e3abde0289922..20f7b20799128 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesService.java
@@ -44,7 +44,8 @@
public class LegacyAwareTopicPoliciesService implements TopicPoliciesService {
private final AsyncLoadingCache isLegacyNamespace;
- private final SystemTopicBasedTopicPoliciesService systemTopicService;
+ @VisibleForTesting
+ final SystemTopicBasedTopicPoliciesService systemTopicService;
private final TopicPoliciesService configuredService;
public LegacyAwareTopicPoliciesService(PulsarService pulsar,
@@ -73,7 +74,10 @@ public CompletableFuture extends Boolean> asyncLoad(NamespaceName key,
@Override
public void start(PulsarService pulsarService) {
- systemTopicService.start(pulsarService);
+ // We should not call `systemTopicService.start()`, which just registers a namespace bundle listener to create
+ // a reader on `/__change_events` when the namespace's bundle is loaded firstly. It's just an
+ // optimization to create the reader before loading any topic. However, it could create a reader on a namespace
+ // that does not even have the __change_events topic.
configuredService.start(pulsarService);
}
@@ -115,12 +119,15 @@ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName,
keepGlobalPoliciesAfterDeleting));
}
+ @Override
+ public CompletableFuture registerListenerAsync(TopicName topicName, TopicPolicyListener listener) {
+ return resolveService(topicName.getNamespaceObject())
+ .thenCompose(service -> service.registerListenerAsync(topicName, listener));
+ }
+
@Override
public boolean registerListener(TopicName topicName, TopicPolicyListener listener) {
- // It's okay to register listeners for both because only one listener will receive the updates
- boolean configuredRegistered = configuredService.registerListener(topicName, listener);
- boolean systemTopicRegistered = systemTopicService.registerListener(topicName, listener);
- return configuredRegistered || systemTopicRegistered;
+ throw new RuntimeException("should not be called");
}
@Override
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java
index 803a18da72d11..5b5fe15723029 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java
@@ -96,6 +96,15 @@ default void start(PulsarService pulsar) {
default void close() throws Exception {
}
+
+ /**
+ * @implNote This method is never called unless by the default implementation of
+ * {@link TopicPoliciesService#registerListenerAsync(TopicName, TopicPolicyListener)}, which is actually called
+ * internally. This method is only retained for backward compatibility on custom implementations.
+ */
+ @Deprecated
+ boolean registerListener(TopicName topicName, TopicPolicyListener listener);
+
/**
* Registers a listener for topic policies updates.
*
@@ -106,10 +115,10 @@ default void close() throws Exception {
* guaranteed to be received by the listener.
* In summary, the listener is guaranteed to receive only the latest value.
*
- *
- * @return true if the listener is registered successfully
*/
- boolean registerListener(TopicName topicName, TopicPolicyListener listener);
+ default CompletableFuture registerListenerAsync(TopicName topicName, TopicPolicyListener listener) {
+ return CompletableFuture.completedFuture(registerListener(topicName, listener));
+ }
/**
* Unregister the topic policies listener.
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 bf795e85c3df7..ed9d0f747f624 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
@@ -4901,7 +4901,10 @@ private void updateSubscriptionsDispatcherRateLimiter() {
protected CompletableFuture initTopicPolicy() {
final var topicPoliciesService = brokerService.pulsar().getTopicPoliciesService();
final var partitionedTopicName = TopicName.getPartitionedTopicName(topic);
- if (topicPoliciesService.registerListener(partitionedTopicName, this)) {
+ return topicPoliciesService.registerListenerAsync(partitionedTopicName, this).thenCompose(registered -> {
+ if (!registered) {
+ return CompletableFuture.completedFuture(null);
+ }
if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) {
return CompletableFuture.completedFuture(null);
}
@@ -4913,8 +4916,7 @@ protected CompletableFuture initTopicPolicy() {
TopicPoliciesService.GetType.LOCAL_ONLY))
.thenAcceptAsync(optionalPolicies -> optionalPolicies.ifPresent(this::onUpdate),
brokerService.getTopicOrderedExecutor());
- }
- return CompletableFuture.completedFuture(null);
+ });
}
@VisibleForTesting
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
index 9d6bfbaf21318..47a7de0528de9 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/LegacyAwareTopicPoliciesServiceTest.java
@@ -26,6 +26,7 @@
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.SystemTopicNames;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.TopicPolicies;
@@ -33,6 +34,7 @@
import org.awaitility.core.ThrowingRunnable;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
+import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
/**
@@ -107,20 +109,28 @@ public void testDowngrade() throws Exception {
assertNull(admin.topicPolicies().getCompactionThreshold(topic2));
}
- @Test
- public void testPoliciesStoredInMetadataStore() throws Exception {
- final var topicName = TopicName.get(metaNamespace + "/test-policies-stored-in-metadata-store");
+ @DataProvider
+ public Object[][] namespaces() {
+ return new Object[][] {
+ { "public/default" },
+ { metaNamespace }
+ };
+ }
+
+ @Test(dataProvider = "namespaces")
+ public void testPoliciesOperations(String namespace) throws Exception {
+ final var topicName = TopicName.get(namespace + "/test-policies-operations");
final var topic = topicName.toString();
admin.topics().createNonPartitionedTopic(topic);
final var compactionThreshold = new AtomicLong(0);
// Verify the exception thrown from one listener does not affect other listeners
- pulsar.getTopicPoliciesService().registerListener(topicName, __ -> {
+ pulsar.getTopicPoliciesService().registerListenerAsync(topicName, __ -> {
throw new RuntimeException("injected failure");
- });
- pulsar.getTopicPoliciesService().registerListener(topicName, policies ->
+ }).get();
+ pulsar.getTopicPoliciesService().registerListenerAsync(topicName, policies ->
Optional.ofNullable(policies).map(TopicPolicies::getCompactionThreshold).ifPresentOrElse(
- compactionThreshold::set, () -> compactionThreshold.set(-1)));
+ compactionThreshold::set, () -> compactionThreshold.set(-1))).get();
// Verify Created events are handled
admin.topicPolicies(false).setCompactionThreshold(topic, 100);
@@ -128,12 +138,16 @@ public void testPoliciesStoredInMetadataStore() throws Exception {
final var localStore = pulsar.getLocalMetadataStore();
final var configurationStore = pulsar.getConfigurationMetadataStore();
- assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, false)).get());
- assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ if (namespace.equals(metaNamespace)) {
+ assertTrue(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, false)).get());
+ assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ }
admin.topicPolicies(true).setCompactionThreshold(topic, 200);
waitUntilAssert(() -> assertEquals(compactionThreshold.get(), 200));
- assertTrue(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ if (namespace.equals(metaNamespace)) {
+ assertTrue(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ }
// Verify Modified events are handled
admin.topicPolicies(false).setCompactionThreshold(topic, 300);
@@ -142,11 +156,17 @@ public void testPoliciesStoredInMetadataStore() throws Exception {
admin.topicPolicies(true).setCompactionThreshold(topic, 400);
waitUntilAssert(() -> assertEquals(compactionThreshold.get(), 400));
+ final var readerNamespaces = ((LegacyAwareTopicPoliciesService) pulsar.getTopicPoliciesService())
+ .systemTopicService.getReaderCaches().keySet();
+ assertFalse(readerNamespaces.contains(NamespaceName.get(metaNamespace)));
+
// Verify Deleted events are handled
admin.topicPolicies(false).deleteTopicPolicies(topic);
waitUntilAssert(() -> assertEquals(compactionThreshold.get(), -1));
- assertFalse(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, false)).get());
- assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ if (namespace.equals(metaNamespace)) {
+ assertFalse(localStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, false)).get());
+ assertFalse(configurationStore.exists(MetadataStoreTopicPoliciesService.pathFor(topicName, true)).get());
+ }
}
@Test
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java
index d02154c81781a..2e8ca8cebec88 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java
@@ -109,7 +109,7 @@ public void onUpdate(TopicPolicies data) {
CompletableFuture f = CompletableFuture.completedFuture(null).thenRunAsync(() -> {
for (int i = 0; i < 100; i++) {
TopicPolicyListener listener = new TopicPolicyListenerImpl();
- systemTopicBasedTopicPoliciesService.registerListener(topicName, listener);
+ systemTopicBasedTopicPoliciesService.registerListenerAsync(topicName, listener);
Assert.assertNotNull(systemTopicBasedTopicPoliciesService.listeners.get(topicName));
Assert.assertTrue(systemTopicBasedTopicPoliciesService.listeners.get(topicName).size() >= 1);
systemTopicBasedTopicPoliciesService.unregisterListener(topicName, listener);
@@ -118,7 +118,7 @@ public void onUpdate(TopicPolicies data) {
for (int i = 0; i < 100; i++) {
TopicPolicyListener listener = new TopicPolicyListenerImpl();
- systemTopicBasedTopicPoliciesService.registerListener(topicName, listener);
+ systemTopicBasedTopicPoliciesService.registerListenerAsync(topicName, listener);
Assert.assertNotNull(systemTopicBasedTopicPoliciesService.listeners.get(topicName));
Assert.assertTrue(systemTopicBasedTopicPoliciesService.listeners.get(topicName).size() >= 1);
systemTopicBasedTopicPoliciesService.unregisterListener(topicName, listener);