diff --git a/conf/broker.conf b/conf/broker.conf index 3c956bdd86dab..e5d8a32e7171c 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1549,6 +1549,16 @@ replicatorPrefix=pulsar.repl # due to missing ZooKeeper watch (disable with value 0) replicationPolicyCheckDurationSeconds=600 +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote +# cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on +# the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for +# remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true + # Default message retention time. # 0 means retention is disabled. -1 means data is not removed by time quota. defaultRetentionTimeInMinutes=0 diff --git a/conf/standalone.conf b/conf/standalone.conf index 635b31ac38def..30b39af8869d4 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -949,6 +949,16 @@ replicationProducerQueueSize=1000 # due to missing ZooKeeper watch (disable with value 0) replicationPolicyCheckDurationSeconds=600 +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote +# cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on +# the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for +# remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true + # Default message retention time. 0 means retention is disabled. -1 means data is not removed by time quota defaultRetentionTimeInMinutes=0 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 26b2f99abf545..72ac85093d0ea 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 @@ -2879,6 +2879,11 @@ public double getLoadBalancerBandwidthOutResourceWeight() { + "inconsistency due to missing ZooKeeper watch (disable with value 0)" ) private int replicationPolicyCheckDurationSeconds = 600; + @FieldContext( + category = CATEGORY_REPLICATION, + doc = "Whether the internal replicator will trigger topic auto-creation on the remote cluster." + ) + private boolean createTopicToRemoteClusterForReplication = true; @Deprecated @FieldContext( category = CATEGORY_REPLICATION, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 1f43aeaa668bc..497af71955158 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -609,11 +609,15 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n .thenCompose(__ -> provisionPartitionedTopicPath(numPartitions, createLocalTopicOnly, properties)) .thenCompose(__ -> tryCreatePartitionsAsync(numPartitions)) .thenRun(() -> { - if (!createLocalTopicOnly && topicName.isGlobal()) { + if (!createLocalTopicOnly && topicName.isGlobal() + && pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { internalCreatePartitionedTopicToReplicatedClustersInBackground(numPartitions); + log.info("[{}] Successfully created partitioned for topic {} for the remote clusters", + clientAppId()); + } else { + log.info("[{}] Skip creating partitioned for topic {} for the remote clusters", + clientAppId(), topicName); } - log.info("[{}] Successfully created partitions for topic {} in cluster {}", - clientAppId(), topicName, pulsar().getConfiguration().getClusterName()); asyncResponse.resume(Response.noContent().build()); }) .exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 747031df7a0af..40e74f83e986d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -3332,6 +3332,12 @@ protected CompletableFuture internalSetReplicationClusters(List cl } return FutureUtil.waitForAll(futures); }).thenCompose(__ -> { + if (!pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { + log.info("[{}] Skip creating partitioned for topic {} for the remote clusters {}", + clientAppId(), topicName, replicationClusters.stream().filter(v -> + !pulsar().getConfig().getClusterName().equals(v)).collect(Collectors.toList())); + return CompletableFuture.completedFuture(null); + } // Sync to create partitioned topic on the remote cluster if needed. TopicName topicNameWithoutPartition = TopicName.get(topicName.getPartitionedTopicName()); return pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 424263720f012..34fd9f17f6ea6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -159,6 +159,10 @@ public String getRemoteCluster() { return remoteCluster; } + protected CompletableFuture prepareCreateProducer() { + return CompletableFuture.completedFuture(null); + } + public void startProducer() { // Guarantee only one task call "producerBuilder.createAsync()". Pair setStartingRes = compareSetAndGetState(State.Disconnected, State.Starting); @@ -185,12 +189,15 @@ public void startProducer() { } log.info("[{}] Starting replicator", replicatorId); + // Force only replicate messages to a non-partitioned topic, to avoid auto-create a partitioned topic on // the remote cluster. - ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; - builderImpl.getConf().setNonPartitionedTopicExpected(true); - producerBuilder.createAsync().thenAccept(producer -> { - setProducerAndTriggerReadEntries(producer); + prepareCreateProducer().thenCompose(ignore -> { + ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; + builderImpl.getConf().setNonPartitionedTopicExpected(true); + return producerBuilder.createAsync().thenAccept(producer -> { + setProducerAndTriggerReadEntries(producer); + }); }).exceptionally(ex -> { Pair setDisconnectedRes = compareSetAndGetState(State.Starting, State.Disconnected); if (setDisconnectedRes.getLeft()) { @@ -215,6 +222,7 @@ public void startProducer() { } return null; }); + } /*** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 1d9df2bcccda3..cd5b2ba721215 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -26,11 +26,13 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.util.FutureUtil; @Slf4j public class GeoPersistentReplicator extends PersistentReplicator { @@ -50,6 +52,33 @@ protected String getProducerName() { return getReplicatorName(replicatorPrefix, localCluster) + REPL_PRODUCER_NAME_DELIMITER + remoteCluster; } + @Override + protected CompletableFuture prepareCreateProducer() { + if (brokerService.getPulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { + return CompletableFuture.completedFuture(null); + } else { + CompletableFuture topicCheckFuture = new CompletableFuture<>(); + replicationClient.getPartitionedTopicMetadata(localTopic.getName(), false, false) + .whenComplete((metadata, ex) -> { + if (ex == null) { + if (metadata.partitions == 0) { + topicCheckFuture.complete(null); + } else { + String errorMsg = String.format("{} Can not create the replicator due to the partitions in the" + + " remote cluster is not 0, but is %s", + replicatorId, metadata.partitions); + log.error(errorMsg); + topicCheckFuture.completeExceptionally( + new PulsarClientException.NotAllowedException(errorMsg)); + } + } else { + topicCheckFuture.completeExceptionally(FutureUtil.unwrapCompletionException(ex)); + } + }); + return topicCheckFuture; + } + } + @Override protected boolean replicateEntries(List entries) { boolean atLeastOneMessageSentForReplication = false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java new file mode 100644 index 0000000000000..0f8db4aaa7316 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java @@ -0,0 +1,208 @@ +/* + * 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.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.Arrays; +import java.util.HashSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class DisabledCreateTopicToRemoteClusterForReplicationTest extends OneWayReplicatorTestBase { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + admin1.namespaces().setRetention(replicatedNamespace, new RetentionPolicies(300, 1024)); + admin2.namespaces().setRetention(replicatedNamespace, new RetentionPolicies(300, 1024)); + admin1.namespaces().setRetention(nonReplicatedNamespace, new RetentionPolicies(300, 1024)); + admin2.namespaces().setRetention(nonReplicatedNamespace, new RetentionPolicies(300, 1024)); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + config.setCreateTopicToRemoteClusterForReplication(false); + config.setReplicationStartAt("earliest"); + } + + @Test + public void testCreatePartitionedTopicWithNsReplication() throws Exception { + String ns = defaultTenant + "/" + UUID.randomUUID().toString().replace("-", ""); + admin1.namespaces().createNamespace(ns); + admin2.namespaces().createNamespace(ns); + admin1.namespaces().setRetention(ns, new RetentionPolicies(3600, -1)); + admin2.namespaces().setRetention(ns, new RetentionPolicies(3600, -1)); + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + final String part1 = TopicName.get(tp).getPartition(0).toString(); + admin1.topics().createPartitionedTopic(tp, 1); + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + + // Trigger and wait for replicator starts. + String msgValue = "msg-1"; + Producer producer1 = client1.newProducer(Schema.STRING).topic(tp).create(); + producer1.send(msgValue); + producer1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createPartitionedTopic(tp, 1); + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(tp).isAckReceiptEnabled(true) + .subscriptionName("s1").subscribe(); + assertEquals(consumer2.receive(10, TimeUnit.SECONDS).getValue(), msgValue); + consumer2.close(); + + // cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1))); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().deletePartitionedTopic(tp, false); + admin2.topics().deletePartitionedTopic(tp, false); + admin1.namespaces().deleteNamespace(ns); + admin2.namespaces().deleteNamespace(ns); + } + + @Test + public void testEnableTopicReplication() throws Exception { + String ns = nonReplicatedNamespace; + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + final String part1 = TopicName.get(tp).getPartition(0).toString(); + admin1.topics().createPartitionedTopic(tp, 1); + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1, cluster2)); + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createPartitionedTopic(tp, 1); + waitReplicatorStarted(part1); + + // cleanup. + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().deletePartitionedTopic(tp, false); + admin2.topics().deletePartitionedTopic(tp, false); + } + + @Test + public void testNonPartitionedTopic() throws Exception { + String ns = nonReplicatedNamespace; + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + admin1.topics().createNonPartitionedTopic(tp); + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1, cluster2)); + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createNonPartitionedTopic(tp); + waitReplicatorStarted(tp); + + // cleanup. + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().delete(tp, false); + admin2.topics().delete(tp, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java index e95b9410f4d12..541408b781be2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java @@ -65,5 +65,6 @@ public void testInitialize() throws Exception { assertEquals(standalone.getConfig().isDispatcherPauseOnAckStatePersistentEnabled(), true); assertEquals(standalone.getConfig().getMaxSecondsToClearTopicNameCache(), 1); assertEquals(standalone.getConfig().getTopicNameCacheMaxCapacity(), 200); + assertEquals(standalone.getConfig().isCreateTopicToRemoteClusterForReplication(), true); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index c64c54d2d191c..77bb36eb68de1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -76,6 +76,7 @@ public void testInit() throws Exception { assertEquals(config.isDispatcherPauseOnAckStatePersistentEnabled(), true); assertEquals(config.getMaxSecondsToClearTopicNameCache(), 1); assertEquals(config.getTopicNameCacheMaxCapacity(), 200); + assertEquals(config.isCreateTopicToRemoteClusterForReplication(), false); OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create(config.getProperties()); assertEquals(offloadPolicies.getManagedLedgerOffloadedReadPriority().getValue(), "bookkeeper-first"); } @@ -293,6 +294,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 1024 * 1024 * 4); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 1); assertEquals(configuration.isDispatcherPauseOnAckStatePersistentEnabled(), false); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), true); } // pulsar_broker_test.conf. try (InputStream inputStream = this.getClass().getClassLoader().getResourceAsStream(fileName)) { @@ -306,6 +308,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 55); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 66); assertEquals(configuration.isDispatcherPauseOnAckStatePersistentEnabled(), true); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), false); } // string input stream. StringBuilder stringBuilder = new StringBuilder(); @@ -318,6 +321,7 @@ public void testTransactionBatchConfigurations() throws Exception{ stringBuilder.append("transactionPendingAckBatchedWriteMaxSize=1025").append(System.lineSeparator()); stringBuilder.append("transactionPendingAckBatchedWriteMaxDelayInMillis=20").append(System.lineSeparator()); stringBuilder.append("dispatcherPauseOnAckStatePersistentEnabled=true").append(System.lineSeparator()); + stringBuilder.append("createTopicToRemoteClusterForReplication=false").append(System.lineSeparator()); try(ByteArrayInputStream inputStream = new ByteArrayInputStream(stringBuilder.toString().getBytes(StandardCharsets.UTF_8))){ configuration = PulsarConfigurationLoader.create(inputStream, ServiceConfiguration.class); @@ -330,6 +334,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 1025); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 20); assertEquals(configuration.isDispatcherPauseOnAckStatePersistentEnabled(), true); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), false); } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index f344a3e3f63da..0fdb29e06866f 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -106,3 +106,4 @@ transactionPendingAckBatchedWriteMaxSize=55 transactionPendingAckBatchedWriteMaxDelayInMillis=66 topicNameCacheMaxCapacity=200 maxSecondsToClearTopicNameCache=1 +createTopicToRemoteClusterForReplication=false diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index c520512e77bf9..d3f9430f29b48 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -97,3 +97,4 @@ maxMessagePublishBufferSizeInMB=-1 dispatcherPauseOnAckStatePersistentEnabled=true topicNameCacheMaxCapacity=200 maxSecondsToClearTopicNameCache=1 +createTopicToRemoteClusterForReplication=true diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index d37c3a10e1607..90cc622dfa04c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -390,7 +390,7 @@ private CompletableFuture checkPartitions(String topic, boolean forceNo getPartitionedTopicMetadata(topic, !forceNoPartitioned, true).thenAccept(metadata -> { if (forceNoPartitioned && metadata.partitions > 0) { String errorMsg = String.format("Can not create the producer[%s] for the topic[%s] that contains %s" - + " partitions, but the producer does not support for a partitioned topic.", + + " partitions b,ut the producer does not support for a partitioned topic.", producerNameForLog, topic, metadata.partitions); log.error(errorMsg); checkPartitions.completeExceptionally(