Skip to content

Commit

Permalink
[fix][broker] Replication stuck when partitions count between two clu…
Browse files Browse the repository at this point in the history
…sters is not the same (apache#22983)

(cherry picked from commit a8ce990)
(cherry picked from commit 25542d8)
  • Loading branch information
poorbarcode authored and nikhil-ctds committed Jul 25, 2024
1 parent f6f2db2 commit 2ed5312
Show file tree
Hide file tree
Showing 8 changed files with 273 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,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 @@ -166,6 +167,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 @@ -150,6 +150,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 @@ -38,10 +38,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 @@ -69,7 +70,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 @@ -85,6 +87,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 @@ -37,6 +37,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
Expand All @@ -47,7 +48,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 @@ -76,11 +79,13 @@
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.util.collections.ConcurrentOpenHashMap;
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;
import org.awaitility.reflect.WhiteboxImpl;
import org.mockito.Mockito;
Expand Down Expand Up @@ -1037,4 +1042,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

0 comments on commit 2ed5312

Please sign in to comment.