Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix][broker] Replication stuck when partitions count between two clusters is not the same #22983

Merged
merged 16 commits into from
Jul 15, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerBuilder;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.impl.ProducerBuilderImpl;
import org.apache.pulsar.client.impl.ProducerImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.common.naming.TopicName;
Expand Down Expand Up @@ -184,6 +185,10 @@ 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);
}).exceptionally(ex -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,12 @@ protected void setProducerAndTriggerReadEntries(Producer<byte[]> producer) {
Pair<Boolean, State> changeStateRes;
changeStateRes = compareSetAndGetState(Starting, Started);
if (changeStateRes.getLeft()) {
if (!(producer instanceof ProducerImpl)) {
log.error("[{}] The partitions count between two clusters is not the same, the replicator can not be"
+ " created successfully: {}", replicatorId, state);
doCloseProducerAsync(producer, () -> {});
throw new ClassCastException(producer.getClass().getName() + " can not be cast to ProducerImpl");
}
this.producer = (ProducerImpl) producer;
HAVE_PENDING_READ_UPDATER.set(this, FALSE);
// Trigger a new read.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,10 +39,11 @@
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerBuilder;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.impl.ConnectionPool;
import org.apache.pulsar.client.impl.ProducerBuilderImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.awaitility.Awaitility;
Expand Down Expand Up @@ -71,7 +72,8 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception {
when(localClient.getCnxPool()).thenReturn(connectionPool);
final PulsarClientImpl remoteClient = mock(PulsarClientImpl.class);
when(remoteClient.getCnxPool()).thenReturn(connectionPool);
final ProducerBuilder producerBuilder = mock(ProducerBuilder.class);
final ProducerConfigurationData producerConf = new ProducerConfigurationData();
final ProducerBuilderImpl producerBuilder = mock(ProducerBuilderImpl.class);
final ConcurrentOpenHashMap<String, CompletableFuture<Optional<Topic>>> topics = new ConcurrentOpenHashMap<>();
when(broker.executor()).thenReturn(eventLoopGroup);
when(broker.getTopics()).thenReturn(topics);
Expand All @@ -87,6 +89,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception {
when(producerBuilder.sendTimeout(anyInt(), any())).thenReturn(producerBuilder);
when(producerBuilder.maxPendingMessages(anyInt())).thenReturn(producerBuilder);
when(producerBuilder.producerName(anyString())).thenReturn(producerBuilder);
when(producerBuilder.getConf()).thenReturn(producerConf);
// Mock create producer fail.
when(producerBuilder.create()).thenThrow(new RuntimeException("mocked ex"));
when(producerBuilder.createAsync())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
Expand All @@ -50,7 +51,9 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.SneakyThrows;
Expand Down Expand Up @@ -79,9 +82,11 @@
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.awaitility.Awaitility;
Expand Down Expand Up @@ -1069,4 +1074,90 @@ public void testConfigReplicationStartAt() throws Exception {
admin1.topics().delete(topic3, false);
admin2.topics().delete(topic3, false);
}

@DataProvider(name = "replicationModes")
public Object[][] replicationModes() {
return new Object[][]{
{ReplicationMode.OneWay},
{ReplicationMode.DoubleWay}
};
}

protected enum ReplicationMode {
OneWay,
DoubleWay;
}

@Test(dataProvider = "replicationModes")
public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception {
String ns = defaultTenant + "/" + UUID.randomUUID().toString().replace("-", "");
admin1.namespaces().createNamespace(ns);
admin2.namespaces().createNamespace(ns);

// Set topic auto-creation rule.
// c1: no-partitioned topic
// c2: partitioned topic with 2 partitions.
AutoTopicCreationOverride autoTopicCreation =
AutoTopicCreationOverrideImpl.builder().allowAutoTopicCreation(true)
.topicType("partitioned").defaultNumPartitions(2).build();
admin2.namespaces().setAutoTopicCreation(ns, autoTopicCreation);
Awaitility.await().untilAsserted(() -> {
assertEquals(admin2.namespaces().getAutoTopicCreationAsync(ns).join().getDefaultNumPartitions(), 2);
// Trigger system topic __change_event's initialize.
pulsar2.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get("persistent://" + ns + "/1"));
});

// Create non-partitioned topic.
// Enable replication.
final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_");
admin1.topics().createNonPartitionedTopic(tp);
admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2)));
if (replicationMode.equals(ReplicationMode.DoubleWay)) {
admin2.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2)));
}

// Trigger and wait for replicator starts.
Producer<String> p1 = client1.newProducer(Schema.STRING).topic(tp).create();
p1.send("msg-1");
p1.close();
Awaitility.await().untilAsserted(() -> {
PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(tp, false).join().get();
assertFalse(persistentTopic.getReplicators().isEmpty());
});

// Verify: the topics are the same between two clusters.
Predicate<String> topicNameFilter = t -> {
TopicName topicName = TopicName.get(t);
if (!topicName.getNamespace().equals(ns)) {
return false;
}
return t.startsWith(tp);
};
Awaitility.await().untilAsserted(() -> {
List<String> topics1 = pulsar1.getBrokerService().getTopics().keys()
.stream().filter(topicNameFilter).collect(Collectors.toList());
List<String> topics2 = pulsar2.getBrokerService().getTopics().keys()
.stream().filter(topicNameFilter).collect(Collectors.toList());
Collections.sort(topics1);
Collections.sort(topics2);
assertEquals(topics1, topics2);
});

// cleanup.
admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1)));
if (replicationMode.equals(ReplicationMode.DoubleWay)) {
admin2.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster2)));
}
Awaitility.await().untilAsserted(() -> {
PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(tp, false).join().get();
assertTrue(persistentTopic.getReplicators().isEmpty());
if (replicationMode.equals(ReplicationMode.DoubleWay)) {
assertTrue(persistentTopic.getReplicators().isEmpty());
}
});
admin1.topics().delete(tp, false);
admin2.topics().delete(tp, false);
admin1.namespaces().deleteNamespace(ns);
admin2.namespaces().deleteNamespace(ns);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -161,4 +161,10 @@ public void testConfigReplicationStartAt() throws Exception {
pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest");
});
}

@Test(enabled = false)
@Override
public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception {
super.testDifferentTopicCreationRule(replicationMode);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/*
* 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.client.api;

import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.client.impl.ProducerBuilderImpl;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
import org.apache.pulsar.common.policies.data.TopicType;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

@Slf4j
public class NonPartitionedTopicExpectedTest extends ProducerConsumerBase {

@BeforeClass
@Override
protected void setup() throws Exception {
super.internalSetup();
super.producerBaseSetup();
}

@AfterClass(alwaysRun = true)
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
}

@Test
public void testWhenNonPartitionedTopicExists() throws Exception {
final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp");
admin.topics().createNonPartitionedTopic(topic);
ProducerBuilderImpl<String> producerBuilder =
(ProducerBuilderImpl<String>) pulsarClient.newProducer(Schema.STRING).topic(topic);
producerBuilder.getConf().setNonPartitionedTopicExpected(true);
// Verify: create successfully.
Producer producer = producerBuilder.create();
// cleanup.
producer.close();
admin.topics().delete(topic, false);
}

@Test
public void testWhenPartitionedTopicExists() throws Exception {
final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp");
admin.topics().createPartitionedTopic(topic, 2);
ProducerBuilderImpl<String> producerBuilder =
(ProducerBuilderImpl<String>) pulsarClient.newProducer(Schema.STRING).topic(topic);
producerBuilder.getConf().setNonPartitionedTopicExpected(true);
// Verify: failed to create.
try {
producerBuilder.create();
Assert.fail("expected an error since producer expected a non-partitioned topic");
} catch (Exception ex) {
// expected an error.
log.error("expected error", ex);
}
// cleanup.
admin.topics().deletePartitionedTopic(topic, false);
}

@DataProvider(name = "topicTypes")
public Object[][] topicTypes() {
return new Object[][]{
{TopicType.PARTITIONED},
{TopicType.NON_PARTITIONED}
};
}

@Test(dataProvider = "topicTypes")
public void testWhenTopicNotExists(TopicType topicType) throws Exception {
final String namespace = "public/default";
final String topic = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp");
final TopicName topicName = TopicName.get(topic);
AutoTopicCreationOverride.Builder policyBuilder = AutoTopicCreationOverride.builder()
.topicType(topicType.toString()).allowAutoTopicCreation(true);
if (topicType.equals(TopicType.PARTITIONED)) {
policyBuilder.defaultNumPartitions(2);
}
AutoTopicCreationOverride policy = policyBuilder.build();
admin.namespaces().setAutoTopicCreation(namespace, policy);

ProducerBuilderImpl<String> producerBuilder =
(ProducerBuilderImpl<String>) pulsarClient.newProducer(Schema.STRING).topic(topic);
producerBuilder.getConf().setNonPartitionedTopicExpected(true);
// Verify: create successfully.
Producer producer = producerBuilder.create();
// Verify: only create non-partitioned topic.
Assert.assertFalse(pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources()
.partitionedTopicExists(topicName));
Assert.assertTrue(pulsar.getNamespaceService().checkNonPartitionedTopicExists(topicName).join());

// cleanup.
producer.close();
admin.topics().delete(topic, false);
admin.namespaces().removeAutoTopicCreation(namespace);
}
}
Loading
Loading