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

[improve][broker] System topic writer/reader connection not counted. #18369

Merged
merged 8 commits into from
Nov 23, 2022
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -438,14 +438,21 @@ private PublishRate publishRateInBroker(ServiceConfiguration config) {
return new PublishRate(config.getMaxPublishRatePerTopicInMessages(), config.getMaxPublishRatePerTopicInBytes());
}

protected boolean isProducersExceeded() {
protected boolean isProducersExceeded(Producer producer) {
if (isSystemTopic() || producer.isRemote()) {
return false;
}
Integer maxProducers = topicPolicies.getMaxProducersPerTopic().get();
if (maxProducers > 0 && maxProducers <= producers.size()) {
if (maxProducers != null && maxProducers > 0 && maxProducers <= getUserCreatedProducerSize()) {
return true;
}
return false;
}

private long getUserCreatedProducerSize() {
return producers.values().stream().filter(p -> !(p.isRemote() || p.getTopic().isSystemTopic())).count();
}
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved

protected void registerTopicPolicyListener() {
if (brokerService.pulsar().getConfig().isSystemTopicEnabled()
&& brokerService.pulsar().getConfig().isTopicLevelPoliciesEnabled()) {
Expand Down Expand Up @@ -960,7 +967,7 @@ protected void checkTopicFenced() throws BrokerServiceException {
}

protected void internalAddProducer(Producer producer) throws BrokerServiceException {
if (isProducersExceeded()) {
if (isProducersExceeded(producer)) {
log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic);
throw new BrokerServiceException.ProducerBusyException("Topic reached max producers limit");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3314,6 +3314,9 @@ public MessageDeduplication getMessageDeduplication() {
}

private boolean checkMaxSubscriptionsPerTopicExceed(String subscriptionName) {
if (isSystemTopic()) {
return false;
}
//Existing subscriptions are not affected
if (StringUtils.isNotEmpty(subscriptionName) && getSubscription(subscriptionName) != null) {
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@
import org.apache.pulsar.client.api.MessageRoutingMode;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.RawMessage;
import org.apache.pulsar.client.api.RawReader;
import org.apache.pulsar.client.api.Schema;
Expand Down Expand Up @@ -1505,4 +1506,36 @@ public void testWhenUpdateReplicationCluster() throws Exception {
assertTrue(topic.getReplicators().isEmpty());
});
}

@Test
public void testReplicatorProducerNotExceed() throws Exception {
log.info("--- testReplicatorProducerNotExceed ---");
String namespace1 = "pulsar/ns11";
admin1.namespaces().createNamespace(namespace1);
admin1.namespaces().setNamespaceReplicationClusters(namespace1, Sets.newHashSet("r1", "r2"));
final TopicName dest1 = TopicName.get(
BrokerTestUtil.newUniqueName("persistent://" + namespace1 + "/testReplicatorProducerNotExceed1"));
String namespace2 = "pulsar/ns22";
admin2.namespaces().createNamespace(namespace2);
admin2.namespaces().setNamespaceReplicationClusters(namespace2, Sets.newHashSet("r1", "r2"));
final TopicName dest2 = TopicName.get(
BrokerTestUtil.newUniqueName("persistent://" + namespace1 + "/testReplicatorProducerNotExceed2"));
admin1.topics().createPartitionedTopic(dest1.toString(), 1);
admin1.topicPolicies().setMaxProducers(dest1.toString(), 1);
admin2.topics().createPartitionedTopic(dest2.toString(), 1);
admin2.topicPolicies().setMaxProducers(dest2.toString(), 1);
@Cleanup
MessageProducer producer1 = new MessageProducer(url1, dest1);
log.info("--- Starting producer1 --- " + url1);

producer1.produce(1);

@Cleanup
MessageProducer producer2 = new MessageProducer(url2, dest2);
log.info("--- Starting producer2 --- " + url2);

producer2.produce(1);
Technoboy- marked this conversation as resolved.
Show resolved Hide resolved

Assert.assertThrows(PulsarClientException.ProducerBusyException.class, () -> new MessageProducer(url2, dest2));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

import lombok.Cleanup;
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
Expand All @@ -47,6 +48,7 @@
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.common.events.PulsarEvent;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.SystemTopicNames;
import org.apache.pulsar.common.naming.TopicName;
Expand Down Expand Up @@ -202,7 +204,7 @@ public void testHeartbeatTopicNotAllowedToSendEvent() throws Exception {
}

@Test
private void testSetBacklogCausedCreatingProducerFailure() throws Exception {
public void testSetBacklogCausedCreatingProducerFailure() throws Exception {
final String ns = "prop/ns-test";
final String topic = ns + "/topic-1";

Expand Down Expand Up @@ -260,4 +262,36 @@ private void testSetBacklogCausedCreatingProducerFailure() throws Exception {
Assert.fail("failed to create producer");
}
}

@Test
public void testSystemTopicNotCheckExceed() throws Exception {
final String ns = "prop/ns-test";
final String topic = ns + "/topic-1";

admin.namespaces().createNamespace(ns, 2);
admin.topics().createPartitionedTopic(String.format("persistent://%s", topic), 1);

admin.topicPolicies().setMaxConsumers(topic, 1);
NamespaceEventsSystemTopicFactory systemTopicFactory = new NamespaceEventsSystemTopicFactory(pulsarClient);
TopicPoliciesSystemTopicClient systemTopicClientForNamespace = systemTopicFactory
.createTopicPoliciesSystemTopicClient(NamespaceName.get(ns));
SystemTopicClient.Reader reader1 = systemTopicClientForNamespace.newReader();
SystemTopicClient.Reader reader2 = systemTopicClientForNamespace.newReader();

admin.topicPolicies().setMaxProducers(topic, 1);

CompletableFuture<SystemTopicClient.Writer<PulsarEvent>> writer1 = systemTopicClientForNamespace.newWriterAsync();
CompletableFuture<SystemTopicClient.Writer<PulsarEvent>> writer2 = systemTopicClientForNamespace.newWriterAsync();
CompletableFuture<Void> f1 = admin.topicPolicies().setCompactionThresholdAsync(topic, 1L);

FutureUtil.waitForAll(List.of(writer1, writer2, f1)).join();
Assert.assertTrue(reader1.hasMoreEvents());
Assert.assertNotNull(reader1.readNext());
Assert.assertTrue(reader2.hasMoreEvents());
Assert.assertNotNull(reader2.readNext());
reader1.close();
reader2.close();
writer1.get().close();
writer2.get().close();
}
}