codelipenghui commented on code in PR #21231:
URL: https://github.com/apache/pulsar/pull/21231#discussion_r1335670651
##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -1760,165 +1760,172 @@ public void openLedgerFailed(ManagedLedgerException
exception, Object ctx) {
});
}
- public CompletableFuture<ManagedLedgerConfig>
getManagedLedgerConfig(TopicName topicName) {
+ public CompletableFuture<ManagedLedgerConfig>
getManagedLedgerConfig(@Nonnull TopicName topicName) {
+ requireNonNull(topicName);
NamespaceName namespace = topicName.getNamespaceObject();
ServiceConfiguration serviceConfig = pulsar.getConfiguration();
NamespaceResources nsr =
pulsar.getPulsarResources().getNamespaceResources();
LocalPoliciesResources lpr =
pulsar.getPulsarResources().getLocalPolicies();
- return nsr.getPoliciesAsync(namespace)
- .thenCombine(lpr.getLocalPoliciesAsync(namespace), (policies,
localPolicies) -> {
- PersistencePolicies persistencePolicies = null;
- RetentionPolicies retentionPolicies = null;
- OffloadPoliciesImpl topicLevelOffloadPolicies = null;
-
- if (pulsar.getConfig().isTopicLevelPoliciesEnabled()
- &&
!NamespaceService.isSystemServiceNamespace(namespace.toString())) {
- final TopicPolicies topicPolicies =
pulsar.getTopicPoliciesService()
- .getTopicPoliciesIfExists(topicName);
- if (topicPolicies != null) {
- persistencePolicies =
topicPolicies.getPersistence();
- retentionPolicies =
topicPolicies.getRetentionPolicies();
- topicLevelOffloadPolicies =
topicPolicies.getOffloadPolicies();
- }
- }
-
- if (persistencePolicies == null) {
- persistencePolicies = policies.map(p ->
p.persistence).orElseGet(
- () -> new
PersistencePolicies(serviceConfig.getManagedLedgerDefaultEnsembleSize(),
-
serviceConfig.getManagedLedgerDefaultWriteQuorum(),
-
serviceConfig.getManagedLedgerDefaultAckQuorum(),
-
serviceConfig.getManagedLedgerDefaultMarkDeleteRateLimit()));
- }
+ final CompletableFuture<Optional<TopicPolicies>> topicPoliciesFuture;
+ if (pulsar.getConfig().isTopicLevelPoliciesEnabled()
+ && !NamespaceService.isSystemServiceNamespace(namespace.toString())
+ &&
!SystemTopicNames.isTopicPoliciesSystemTopic(topicName.toString())) {
+ topicPoliciesFuture =
pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName);
+ } else {
+ topicPoliciesFuture =
CompletableFuture.completedFuture(Optional.empty());
+ }
+ return topicPoliciesFuture.thenCompose(topicPoliciesOptional -> {
+ final CompletableFuture<Optional<Policies>> nsPolicies =
nsr.getPoliciesAsync(namespace);
+ final CompletableFuture<Optional<LocalPolicies>> lcPolicies =
lpr.getLocalPoliciesAsync(namespace);
+ return nsPolicies.thenCombine(lcPolicies, (policies,
localPolicies) -> {
+ PersistencePolicies persistencePolicies = null;
+ RetentionPolicies retentionPolicies = null;
+ OffloadPoliciesImpl topicLevelOffloadPolicies = null;
+ if (topicPoliciesOptional.isPresent()) {
+ final TopicPolicies topicPolicies =
topicPoliciesOptional.get();
+ persistencePolicies = topicPolicies.getPersistence();
+ retentionPolicies = topicPolicies.getRetentionPolicies();
+ topicLevelOffloadPolicies =
topicPolicies.getOffloadPolicies();
+ }
Review Comment:
If the topic policy is present, it doesn't mean the retention policy,
persistence policy, or offload policy is also present? Users might configure
other policies like rate limit or something.
##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java:
##########
@@ -265,39 +296,48 @@ public CompletableFuture<TopicPolicies>
getTopicPoliciesBypassCacheAsync(TopicNa
@Override
public CompletableFuture<Void>
addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) {
- CompletableFuture<Void> result = new CompletableFuture<>();
NamespaceName namespace = namespaceBundle.getNamespaceObject();
if (NamespaceService.isHeartbeatNamespace(namespace)) {
- result.complete(null);
- return result;
+ return CompletableFuture.completedFuture(null);
}
synchronized (this) {
if (readerCaches.get(namespace) != null) {
ownedBundlesCountPerNamespace.get(namespace).incrementAndGet();
- result.complete(null);
+ return CompletableFuture.completedFuture(null);
} else {
- prepareInitPoliciesCache(namespace, result);
+ return prepareInitPoliciesCacheAsync(namespace);
}
}
- return result;
}
- private void prepareInitPoliciesCache(@Nonnull NamespaceName namespace,
CompletableFuture<Void> result) {
- if (policyCacheInitMap.putIfAbsent(namespace, false) == null) {
- CompletableFuture<SystemTopicClient.Reader<PulsarEvent>>
readerCompletableFuture =
+ private @Nonnull CompletableFuture<Void>
prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) {
+ requireNonNull(namespace);
+ return policyCacheInitMap.computeIfAbsent(namespace, (k) -> {
+ final CompletableFuture<SystemTopicClient.Reader<PulsarEvent>>
readerCompletableFuture =
createSystemTopicClientWithRetry(namespace);
readerCaches.put(namespace, readerCompletableFuture);
ownedBundlesCountPerNamespace.putIfAbsent(namespace, new
AtomicInteger(1));
- readerCompletableFuture.thenAccept(reader -> {
- initPolicesCache(reader, result);
- result.thenRun(() -> readMorePolicies(reader));
- }).exceptionally(ex -> {
- log.error("[{}] Failed to create reader on __change_events
topic", namespace, ex);
- cleanCacheAndCloseReader(namespace, false);
- result.completeExceptionally(ex);
+ final CompletableFuture<Void> initFuture = readerCompletableFuture
+ .thenCompose(reader -> {
+ final CompletableFuture<Void> stageFuture = new
CompletableFuture<>();
+ initPolicesCache(reader, stageFuture);
+ return stageFuture
+ // Read policies in background
+ .thenAccept(__ ->
readMorePoliciesAsync(reader));
+ });
+ initFuture.exceptionally(ex -> {
+ try {
+ log.error("[{}] Failed to create reader on __change_events
topic", namespace, ex);
+ cleanCacheAndCloseReader(namespace, false);
Review Comment:
https://github.com/apache/pulsar/pull/21231/files#diff-9d2948d863c111e4be6d508a1c573667a1326b98c4314e917ba9e344bb61dc27R457-R460
It will put a null value to `policyCacheInitMap`. Do we need to remove it
directly? Otherwise, prepareInitPoliciesCacheAsync will get a chance to return
null?
##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesWithBrokerRestartTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.common.policies.data.RetentionPolicies;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.TimeUnit;
+
+@Slf4j
+@Test(groups = "broker-admin")
+public class TopicPoliciesWithBrokerRestartTest extends
MockedPulsarServiceBaseTest {
+
+ @Override
+ @BeforeClass(alwaysRun = true)
+ protected void setup() throws Exception {
+ super.internalSetup();
+ setupDefaultTenantAndNamespace();
+ }
+
+ @Override
+ @AfterClass(alwaysRun = true)
+ protected void cleanup() throws Exception {
+ super.internalCleanup();
+ }
+
+
+ @Test
+ public void testRetentionWithBrokerRestart() throws Exception {
+ final int messages = 1_000;
+ // (1) Init topic
+ admin.namespaces().createNamespace("public/retention");
+ final String topicName =
"persistent://public/retention/retention_with_broker_restart";
+ admin.topics().createNonPartitionedTopic(topicName);
+ for (int i = 0; i < 500; i++) {
+ final String shadowTopicNames = topicName + "_" + i;
+ admin.topics().createNonPartitionedTopic(shadowTopicNames);
+ }
+ // (2) Set retention
+ final RetentionPolicies retentionPolicies = new RetentionPolicies(20,
20);
+ for (int i = 0; i < 500; i++) {
+ final String shadowTopicNames = topicName + "_" + i;
+ admin.topicPolicies().setRetention(shadowTopicNames,
retentionPolicies);
+ }
+ admin.topicPolicies().setRetention(topicName, retentionPolicies);
+ // (3) Send messages
+ @Cleanup
+ final Producer<byte[]> publisher = pulsarClient.newProducer()
+ .topic(topicName)
+ .create();
+ for (int i = 0; i < messages; i++) {
+ publisher.send((i + "").getBytes(StandardCharsets.UTF_8));
+ }
+ // (4) Check configuration
+ Awaitility.await().untilAsserted(() -> {
+ final PersistentTopic persistentTopic1 = (PersistentTopic)
+ pulsar.getBrokerService().getTopic(topicName,
true).join().get();
+ final ManagedLedgerImpl managedLedger1 = (ManagedLedgerImpl)
persistentTopic1.getManagedLedger();
+
Assert.assertEquals(managedLedger1.getConfig().getRetentionSizeInMB(), 20);
+
Assert.assertEquals(managedLedger1.getConfig().getRetentionTimeMillis(),
+ TimeUnit.MINUTES.toMillis(20));
+ });
+ // (5) Restart broker
+ restartBroker();
+ // (6) Check configuration again
+ admin.lookups().lookupTopic(topicName);
+ final PersistentTopic persistentTopic2 = (PersistentTopic)
+ pulsar.getBrokerService().getTopic(topicName,
true).join().get();
+ final ManagedLedgerImpl managedLedger2 = (ManagedLedgerImpl)
persistentTopic2.getManagedLedger();
+ Assert.assertEquals(managedLedger2.getConfig().getRetentionSizeInMB(),
20);
+
Assert.assertEquals(managedLedger2.getConfig().getRetentionTimeMillis(),
+ TimeUnit.MINUTES.toMillis(20));
Review Comment:
You have created 500 topics, why not check all of them?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]