sijie closed pull request #2977: Fix-2876 Limit the client reconnect behavior URL: https://github.com/apache/pulsar/pull/2977
This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/conf/broker.conf b/conf/broker.conf index ad6135f368..186b6d77c5 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -148,6 +148,15 @@ maxUnackedMessagesPerBroker=0 # limit/2 messages maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 +# Too many subscribe requests from a consumer can cause broker rewinding consumer cursors and loading data from bookies, +# hence causing high network bandwidth usage +# When the positive value is set, broker will throttle the subscribe requests for one consumer. +# Otherwise, the throttling will be disabled. The default value of this setting is 0 - throttling is disabled. +subscribeThrottlingRatePerConsumer=0 + +# Rate period for {subscribeThrottlingRatePerConsumer}. Default is 30s. +subscribeRatePeriodPerConsumerInSecond=30 + # Default messages per second dispatch throttling-limit for every topic. Using a value of 0, is disabling default # message dispatch-throttling dispatchThrottlingRatePerTopicInMsg=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 29fc54a242..2a7b16edad 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 @@ -157,6 +157,15 @@ // than this percentage limit and subscription will not receive any new messages until that subscription acks back // limit/2 messages private double maxUnackedMessagesPerSubscriptionOnBrokerBlocked = 0.16; + // Too many subscribe requests from a consumer can cause broker rewinding consumer cursors and loading data from bookies, + // hence causing high network bandwidth usage + // When the positive value is set, broker will throttle the subscribe requests for one consumer. + // Otherwise, the throttling will be disabled. The default value of this setting is 0 - throttling is disabled. + @FieldContext(dynamic = true) + private int subscribeThrottlingRatePerConsumer = 0; + // Rate period for {subscribeThrottlingRatePerConsumer}. Default is 30s. + @FieldContext(minValue = 1, dynamic = true) + private int subscribeRatePeriodPerConsumerInSecond = 30; // Default number of message dispatching throttling-limit for every topic. Using a value of 0, is disabling default // message dispatch-throttling @FieldContext(dynamic = true) @@ -1796,4 +1805,20 @@ public int getBrokerServiceCompactionMonitorIntervalInSeconds() { public void setBacklogQuotaDefaultRetentionPolicy(BacklogQuota.RetentionPolicy backlogQuotaDefaultRetentionPolicy) { this.backlogQuotaDefaultRetentionPolicy = backlogQuotaDefaultRetentionPolicy; } + + public int getSubscribeThrottlingRatePerConsumer() { + return subscribeThrottlingRatePerConsumer; + } + + public void setSubscribeThrottlingRatePerConsumer(int subscribeThrottlingRatePerConsumer) { + this.subscribeThrottlingRatePerConsumer = subscribeThrottlingRatePerConsumer; + } + + public int getSubscribeRatePeriodPerConsumerInSecond() { + return subscribeRatePeriodPerConsumerInSecond; + } + + public void setSubscribeRatePeriodPerConsumerInSecond(int subscribeRatePeriodPerConsumerInSecond) { + this.subscribeRatePeriodPerConsumerInSecond = subscribeRatePeriodPerConsumerInSecond; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index a00535430a..105ca25de3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -74,6 +74,7 @@ import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; import org.apache.pulsar.common.util.FutureUtil; @@ -696,6 +697,55 @@ protected DispatchRate internalGetSubscriptionDispatchRate() { } } + protected void internalSetSubscribeRate(SubscribeRate subscribeRate) { + log.info("[{}] Set namespace subscribe-rate {}/{}", clientAppId(), namespaceName, subscribeRate); + validateSuperUserAccess(); + + Entry<Policies, Stat> policiesNode = null; + + try { + final String path = path(POLICIES, namespaceName.toString()); + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist")); + policiesNode.getKey().clusterSubscribeRate.put(pulsar().getConfiguration().getClusterName(), subscribeRate); + + // Write back the new policies into zookeeper + globalZk().setData(path, jsonMapper().writeValueAsBytes(policiesNode.getKey()), + policiesNode.getValue().getVersion()); + policiesCache().invalidate(path); + + log.info("[{}] Successfully updated the subscribeRate for cluster on namespace {}", clientAppId(), + namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update the subscribeRate for cluster on namespace {}: does not exist", + clientAppId(), namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to update the subscribeRate for cluster on namespace {} expected policy node version={} : concurrent modification", + clientAppId(), namespaceName, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update the subscribeRate for cluster on namespace {}", clientAppId(), + namespaceName, e); + throw new RestException(e); + } + } + + protected SubscribeRate internalGetSubscribeRate() { + validateAdminAccessForTenant(namespaceName.getTenant()); + Policies policies = getNamespacePolicies(namespaceName); + SubscribeRate subscribeRate = policies.clusterSubscribeRate.get(pulsar().getConfiguration().getClusterName()); + if (subscribeRate != null) { + return subscribeRate; + } else { + throw new RestException(Status.NOT_FOUND, + "Subscribe-rate is not configured for cluster " + pulsar().getConfiguration().getClusterName()); + } + } + protected void internalSetBacklogQuota(BacklogQuotaType backlogQuotaType, BacklogQuota backlogQuota) { validateAdminAccessForTenant(namespaceName.getTenant()); validatePoliciesReadOnlyAccess(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index e335521053..58686d8e53 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -45,6 +45,7 @@ import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; import org.slf4j.Logger; @@ -345,6 +346,27 @@ public DispatchRate getSubscriptionDispatchRate(@PathParam("tenant") String tena return internalGetSubscriptionDispatchRate(); } + @POST + @Path("/{tenant}/{namespace}/subscribeRate") + @ApiOperation(value = "Set subscribe-rate throttling for all topics of the namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void setSubscribeRate(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, + SubscribeRate subscribeRate) { + validateNamespaceName(tenant, namespace); + internalSetSubscribeRate(subscribeRate); + } + + @GET + @Path("/{tenant}/{namespace}/subscribeRate") + @ApiOperation(value = "Get subscribe-rate configured for the namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public SubscribeRate getSubscribeRate(@PathParam("tenant") String tenant, + @PathParam("namespace") String namespace) { + validateNamespaceName(tenant, namespace); + return internalGetSubscribeRate(); + } + @GET @Path("/{tenant}/{namespace}/backlogQuotaMap") @ApiOperation(value = "Get backlog quota map on a namespace.") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 483bf66451..181c74cfbd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -1410,6 +1410,10 @@ public State getState() { return state; } + public SocketAddress getRemoteAddress() { + return remoteAddress; + } + public BrokerService getBrokerService() { return service; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index d0be156b70..eed5b27bbb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -168,6 +168,7 @@ // doesn't support batch-message private volatile boolean hasBatchMessagePublished = false; private final DispatchRateLimiter dispatchRateLimiter; + private final SubscribeRateLimiter subscribeRateLimiter; public static final int MESSAGE_RATE_BACKOFF_MS = 1000; private final MessageDeduplication messageDeduplication; @@ -226,6 +227,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS USAGE_COUNT_UPDATER.set(this, 0); this.dispatchRateLimiter = new DispatchRateLimiter(this); + this.subscribeRateLimiter = new SubscribeRateLimiter(this); this.compactedTopic = new CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient()); @@ -494,6 +496,18 @@ public void removeProducer(Producer producer) { return future; } + if (cnx.getRemoteAddress() != null && cnx.getRemoteAddress().toString().contains(":")) { + SubscribeRateLimiter.ConsumerIdentifier consumer = new SubscribeRateLimiter.ConsumerIdentifier( + cnx.getRemoteAddress().toString().split(":")[0], consumerName, consumerId); + if (!subscribeRateLimiter.subscribeAvailable(consumer) || !subscribeRateLimiter.tryAcquire(consumer)) { + log.warn("[{}] Failed to create subscription for {} {} limited by {}, available {}", + topic, subscriptionName, consumer, subscribeRateLimiter.getSubscribeRate(), + subscribeRateLimiter.getAvailableSubscribeRateLimit(consumer)); + future.completeExceptionally(new NotAllowedException("Subscribe limited by subscribe rate limit per consumer.")); + return future; + } + } + lock.readLock().lock(); try { if (isFenced) { @@ -840,6 +854,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { }, null); dispatchRateLimiter.close(); + subscribeRateLimiter.close(); }).exceptionally(exception -> { log.error("[{}] Error closing topic", topic, exception); @@ -1580,6 +1595,7 @@ private boolean shouldTopicBeRetained() { CompletableFuture<Void> dedupFuture = checkDeduplicationStatus(); CompletableFuture<Void> persistentPoliciesFuture = checkPersistencePolicies(); dispatchRateLimiter.onPoliciesUpdate(data); + subscribeRateLimiter.onPoliciesUpdate(data); return CompletableFuture.allOf(replicationFuture, dedupFuture, persistentPoliciesFuture); } @@ -1727,6 +1743,10 @@ public DispatchRateLimiter getDispatchRateLimiter() { return this.dispatchRateLimiter; } + public SubscribeRateLimiter getSubscribeRateLimiter() { + return this.subscribeRateLimiter; + } + public long getLastPublishedSequenceId(String producerName) { return messageDeduplication.getLastPublishedSequenceId(producerName); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java new file mode 100644 index 0000000000..d6c65d6f5f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -0,0 +1,283 @@ +/** + * 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.persistent; + + +import com.google.common.base.MoreObjects; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.SubscribeRate; +import org.apache.pulsar.common.util.RateLimiter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.apache.pulsar.broker.web.PulsarWebResource.path; +import static org.apache.pulsar.zookeeper.ZooKeeperCache.cacheTimeOutInSec; + +public class SubscribeRateLimiter { + + private final String topicName; + private final BrokerService brokerService; + private ConcurrentHashMap<ConsumerIdentifier, RateLimiter> subscribeRateLimiter; + private final ScheduledExecutorService executorService; + private ScheduledFuture<?> resetTask; + private SubscribeRate subscribeRate; + + public SubscribeRateLimiter(PersistentTopic topic) { + this.topicName = topic.getName(); + this.brokerService = topic.getBrokerService(); + subscribeRateLimiter = new ConcurrentHashMap<>(); + this.executorService = brokerService.pulsar().getExecutor(); + this.subscribeRate = getPoliciesSubscribeRate(); + if (this.subscribeRate == null) { + this.subscribeRate = new SubscribeRate(brokerService.pulsar().getConfiguration().getSubscribeThrottlingRatePerConsumer(), + brokerService.pulsar().getConfiguration().getSubscribeRatePeriodPerConsumerInSecond()); + + } + if (isSubscribeRateEnabled(this.subscribeRate)) { + resetTask = createTask(); + log.info("[{}] [{}] configured subscribe-dispatch rate at broker {}", this.topicName, subscribeRate); + } + } + + /** + * returns available subscribes if subscribe-throttling is enabled else it returns -1 + * + * @return + */ + public long getAvailableSubscribeRateLimit(ConsumerIdentifier consumerIdentifier) { + return subscribeRateLimiter.get(consumerIdentifier) == null ? -1 : subscribeRateLimiter.get(consumerIdentifier).getAvailablePermits(); + } + + /** + * It acquires subscribe from subscribe-limiter and returns if acquired permits succeed. + * + * @return + */ + public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { + addSubscribeLimiterIfAbsent(consumerIdentifier); + return subscribeRateLimiter.get(consumerIdentifier) == null || subscribeRateLimiter.get(consumerIdentifier).tryAcquire(); + } + + /** + * checks if subscribe-rate limit is configured and if it's configured then check if subscribe are available or not. + * + * @return + */ + public boolean subscribeAvailable(ConsumerIdentifier consumerIdentifier) { + return (subscribeRateLimiter.get(consumerIdentifier) == null|| subscribeRateLimiter.get(consumerIdentifier).getAvailablePermits() > 0); + } + + /** + * Update subscribe-throttling-rate. gives first priority to namespace-policy configured subscribe rate else applies + * default broker subscribe-throttling-rate + */ + private synchronized void addSubscribeLimiterIfAbsent(ConsumerIdentifier consumerIdentifier) { + if (subscribeRateLimiter.get(consumerIdentifier) != null) { + return; + } + updateSubscribeRate(consumerIdentifier, this.subscribeRate); + } + + private synchronized void removeSubscribeLimiter(ConsumerIdentifier consumerIdentifier) { + if (this.subscribeRateLimiter.get(consumerIdentifier) != null) { + this.subscribeRateLimiter.get(consumerIdentifier).close(); + this.subscribeRateLimiter.remove(consumerIdentifier); + } + } + + /** + * Update subscribe rate by updating rate-limiter. If subscribe-rate is configured < 0 then it closes + * the rate-limiter and disables appropriate rate-limiter. + * + * @param subscribeRate + */ + private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentifier, SubscribeRate subscribeRate) { + + long ratePerConsumer = subscribeRate.subscribeThrottlingRatePerConsumer; + long ratePeriod = subscribeRate.ratePeriodInSecond; + + // update subscribe-rateLimiter + if (ratePerConsumer > 0) { + if (this.subscribeRateLimiter.get(consumerIdentifier) == null) { + this.subscribeRateLimiter.put(consumerIdentifier, new RateLimiter(brokerService.pulsar().getExecutor(), ratePerConsumer, + ratePeriod, TimeUnit.SECONDS)); + } else { + this.subscribeRateLimiter.get(consumerIdentifier).setRate(ratePerConsumer, ratePeriod, TimeUnit.SECONDS); + } + } else { + // subscribe-rate should be disable and close + removeSubscribeLimiter(consumerIdentifier); + } + } + + public void onPoliciesUpdate(Policies data) { + + String cluster = brokerService.pulsar().getConfiguration().getClusterName(); + + SubscribeRate subscribeRate = data.clusterSubscribeRate.get(cluster); + + // update dispatch-rate only if it's configured in policies else ignore + if (subscribeRate != null) { + final SubscribeRate newSubscribeRate = new SubscribeRate( + brokerService.pulsar().getConfiguration().getSubscribeThrottlingRatePerConsumer(), + brokerService.pulsar().getConfiguration().getSubscribeRatePeriodPerConsumerInSecond() + ); + // if policy-throttling rate is disabled and cluster-throttling is enabled then apply + // cluster-throttling rate + if (!isSubscribeRateEnabled(subscribeRate) && isSubscribeRateEnabled(newSubscribeRate)) { + subscribeRate = newSubscribeRate; + } + this.subscribeRate = subscribeRate; + stopResetTask(); + for (ConsumerIdentifier consumerIdentifier : this.subscribeRateLimiter.keySet()) { + updateSubscribeRate(consumerIdentifier, subscribeRate); + } + if (isSubscribeRateEnabled(this.subscribeRate)) { + this.resetTask = createTask(); + log.info("[{}] [{}] configured subscribe-dispatch rate at broker {}", this.topicName, subscribeRate); + } + } + } + + /** + * Gets configured subscribe-rate from namespace policies. Returns null if subscribe-rate is not configured + * + * @return + */ + public SubscribeRate getPoliciesSubscribeRate() { + final NamespaceName namespace = TopicName.get(this.topicName).getNamespaceObject(); + final String cluster = brokerService.pulsar().getConfiguration().getClusterName(); + final String path = path(POLICIES, namespace.toString()); + Optional<Policies> policies = Optional.empty(); + try { + policies = brokerService.pulsar().getConfigurationCache().policiesCache().getAsync(path) + .get(cacheTimeOutInSec, SECONDS); + } catch (Exception e) { + log.warn("Failed to get subscribe-rate for {}", this.topicName, e); + } + // return policy-subscribe rate only if it's enabled in policies + return policies.map(p -> { + if (p.clusterSubscribeRate != null) { + SubscribeRate subscribeRate = p.clusterSubscribeRate.get(cluster); + return isSubscribeRateEnabled(subscribeRate) ? subscribeRate : null; + } else { + return null; + } + + }).orElse(null); + } + + /** + * Get configured msg subscribe-throttling rate. Returns -1 if not configured + * + * @return + */ + public long getSubscribeRatePerConsumer(ConsumerIdentifier consumerIdentifier) { + return subscribeRateLimiter.get(consumerIdentifier) != null ? subscribeRateLimiter.get(consumerIdentifier).getRate() : -1; + } + + private boolean isSubscribeRateEnabled(SubscribeRate subscribeRate) { + return subscribeRate != null && (subscribeRate.subscribeThrottlingRatePerConsumer > 0); + } + + public void close() { + closeAndClearRateLimiters(); + stopResetTask(); + } + + private ScheduledFuture<?> createTask() { + return executorService.scheduleAtFixedRate(this::closeAndClearRateLimiters, + this.subscribeRate.ratePeriodInSecond, + this.subscribeRate.ratePeriodInSecond, + TimeUnit.SECONDS); + } + + private void stopResetTask() { + if (this.resetTask != null) { + this.resetTask.cancel(false); + } + } + + private synchronized void closeAndClearRateLimiters() { + // close rate-limiter + this.subscribeRateLimiter.values().forEach(rateLimiter -> { + if (rateLimiter != null) { + rateLimiter.close(); + } + }); + this.subscribeRateLimiter.clear(); + } + + public SubscribeRate getSubscribeRate() { + return subscribeRate; + } + + public static class ConsumerIdentifier { + + private String host; + + private String consumerName; + + private long consumerId; + + public ConsumerIdentifier(String host, String consumerName, long consumerId) { + this.host = host; + this.consumerName = consumerName; + this.consumerId = consumerId; + } + + @Override + public int hashCode() { + return Objects.hash(host, consumerName, consumerId); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ConsumerIdentifier) { + ConsumerIdentifier consumer = (ConsumerIdentifier) obj; + return Objects.equals(host, consumer.host) + && Objects.equals(consumerName, consumer.consumerName) + && Objects.equals(consumerId, consumer.consumerId); + } + return false; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("host", host) + .add("consumerName", consumerName) + .add("consumerId", consumerId).toString(); + } + } + + private static final Logger log = LoggerFactory.getLogger(SubscribeRateLimiter.class); +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index 01ff0e3c4f..ffc1ab416c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -28,6 +28,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -56,7 +57,9 @@ import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; 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.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundles; import org.apache.pulsar.common.naming.NamespaceName; @@ -66,6 +69,7 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -75,6 +79,8 @@ import org.hamcrest.Description; import org.hamcrest.Matcher; import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -1153,6 +1159,47 @@ public void testDeleteNamespace() throws Exception { } + @Test + public void testSubscribeRate() throws Exception { + SubscribeRate subscribeRate = new SubscribeRate(1, 5); + String namespace = "my-tenants/my-namespace"; + admin.tenants().createTenant("my-tenants", new TenantInfo(Sets.newHashSet(), Sets.newHashSet(testLocalCluster))); + admin.namespaces().createNamespace(namespace, Sets.newHashSet(testLocalCluster)); + admin.namespaces().setSubscribeRate(namespace, subscribeRate); + assertEquals(subscribeRate, admin.namespaces().getSubscribeRate(namespace)); + String topicName = "persistent://" + namespace + "/" + "subscribe-rate"; + + admin.topics().createPartitionedTopic(topicName, 2); + pulsar.getConfiguration().setAuthorizationEnabled(false); + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionType(SubscriptionType.Shared) + .subscriptionName("subscribe-rate") + .subscribe(); + assertTrue(consumer.isConnected()); + + // Subscribe Rate Limiter is enabled, will limited by broker + pulsarClient.updateServiceUrl(lookupUrl.toString()); + Thread.sleep(1000L); + assertFalse(consumer.isConnected()); + + // Out of limit period + Thread.sleep(6000L); + pulsarClient.updateServiceUrl(lookupUrl.toString()); + assertTrue(consumer.isConnected()); + + // Disable Subscribe Rate Limiter + subscribeRate = new SubscribeRate(0, 10); + admin.namespaces().setSubscribeRate(namespace, subscribeRate); + pulsarClient.updateServiceUrl(lookupUrl.toString()); + Thread.sleep(1000L); + assertTrue(consumer.isConnected()); + pulsar.getConfiguration().setAuthorizationEnabled(true); + admin.topics().deletePartitionedTopic(topicName, true); + admin.namespaces().deleteNamespace(namespace); + admin.tenants().deleteTenant("my-tenants"); + } + private void mockWebUrl(URL localWebServiceUrl, NamespaceName namespace) throws Exception { doReturn(Optional.of(localWebServiceUrl)).when(nsSvc) .getWebServiceUrl(Mockito.argThat(new Matcher<NamespaceBundle>() { @@ -1192,4 +1239,6 @@ public void _dont_implement_Matcher___instead_extend_BaseMatcher_() { } })); } + + private static final Logger log = LoggerFactory.getLogger(NamespacesTest.class); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java index 6460c1ff5c..cbd5506939 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java @@ -35,6 +35,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; /** @@ -855,7 +856,7 @@ * Set message-dispatch-rate (topics under this namespace can dispatch this many messages per second) * * @param namespace - * @param messageRate + * @param dispatchRate * number of messages per second * @throws PulsarAdminException * Unexpected error @@ -872,6 +873,26 @@ */ DispatchRate getDispatchRate(String namespace) throws PulsarAdminException; + /** + * Set namespace-subscribe-rate (topics under this namespace will limit by subscribeRate) + * + * @param namespace + * @param subscribeRate + * consumer subscribe limit by this subscribeRate + * @throws PulsarAdminException + * Unexpected error + */ + void setSubscribeRate(String namespace, SubscribeRate subscribeRate) throws PulsarAdminException; + + /** Get namespace-subscribe-rate (topics under this namespace allow subscribe times per consumer in a period) + * + * @param namespace + * @returns subscribeRate + * @throws PulsarAdminException + * Unexpected error + */ + SubscribeRate getSubscribeRate(String namespace) throws PulsarAdminException; + /** * Set subscription-message-dispatch-rate (subscriptions under this namespace can dispatch this many messages per second) * diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index c9845a918c..833c2c3efe 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -44,6 +44,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; public class NamespacesImpl extends BaseResource implements Namespaces { @@ -480,6 +481,28 @@ public DispatchRate getDispatchRate(String namespace) throws PulsarAdminExceptio } } + @Override + public void setSubscribeRate(String namespace, SubscribeRate subscribeRate) throws PulsarAdminException { + try { + NamespaceName ns = NamespaceName.get(namespace); + WebTarget path = namespacePath(ns, "subscribeRate"); + request(path).post(Entity.entity(subscribeRate, MediaType.APPLICATION_JSON), ErrorData.class); + } catch (Exception e) { + throw getApiException(e); + } + } + + @Override + public SubscribeRate getSubscribeRate(String namespace) throws PulsarAdminException { + try { + NamespaceName ns = NamespaceName.get(namespace); + WebTarget path = namespacePath(ns, "subscribeRate"); + return request(path).get(SubscribeRate.class); + } catch (Exception e) { + throw getApiException(e); + } + } + @Override public void setSubscriptionDispatchRate(String namespace, DispatchRate dispatchRate) throws PulsarAdminException { try { diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index cc1654bc8c..3b724f2ea9 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -42,6 +42,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; @Parameters(commandDescription = "Operations about namespaces") @@ -462,6 +463,41 @@ void run() throws PulsarAdminException { } } + @Parameters(commandDescription = "Set subscribe-rate per consumer for all topics of the namespace") + private class SetSubscribeRate extends CliCommand { + + @Parameter(description = "tenant/namespace/\n", required = true) + private java.util.List<String> params; + + @Parameter(names = { "--subscribe-rate", + "-sr" }, description = "subscribe-rate (default -1 will be overwrite if not passed)\n", required = false) + private int subscribeRate = -1; + + @Parameter(names = { "--subscribe-rate-period", + "-st" }, description = "subscribe-rate-period in second type (default 30 second will be overwrite if not passed)\n", required = false) + private int subscribeRatePeriodSec = 30; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + admin.namespaces().setSubscribeRate(namespace, + new SubscribeRate(subscribeRate, subscribeRatePeriodSec)); + } + } + + @Parameters(commandDescription = "Get configured subscribe-rate per consumer for all topics of the namespace") + private class GetSubscribeRate extends CliCommand { + @Parameter(description = "tenant/namespace\n", required = true) + private java.util.List<String> params; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + print(admin.namespaces().getSubscribeRate(namespace)); + } + } + + @Parameters(commandDescription = "Set subscription message-dispatch-rate for all subscription of the namespace") private class SetSuscriptionDispatchRate extends CliCommand { @Parameter(description = "tenant/namespace/\n", required = true) @@ -973,6 +1009,9 @@ public CmdNamespaces(PulsarAdmin admin) { jcommander.addCommand("set-dispatch-rate", new SetDispatchRate()); jcommander.addCommand("get-dispatch-rate", new GetDispatchRate()); + jcommander.addCommand("set-subscribe-rate", new SetSubscribeRate()); + jcommander.addCommand("get-subscribe-rate", new GetSubscribeRate()); + jcommander.addCommand("clear-backlog", new ClearBacklog()); jcommander.addCommand("unsubscribe", new Unsubscribe()); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index d376a86812..3d12545a27 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -36,6 +36,7 @@ public Map<BacklogQuota.BacklogQuotaType, BacklogQuota> backlog_quota_map = Maps.newHashMap(); public Map<String, DispatchRate> clusterDispatchRate = Maps.newHashMap(); public Map<String, DispatchRate> subscriptionDispatchRate = Maps.newHashMap(); + public Map<String, SubscribeRate> clusterSubscribeRate = Maps.newHashMap(); public PersistencePolicies persistence = null; // If set, it will override the broker settings for enabling deduplication @@ -72,6 +73,7 @@ public boolean equals(Object obj) { && Objects.equals(replication_clusters, other.replication_clusters) && Objects.equals(backlog_quota_map, other.backlog_quota_map) && Objects.equals(clusterDispatchRate, other.clusterDispatchRate) + && Objects.equals(clusterSubscribeRate, other.clusterSubscribeRate) && Objects.equals(deduplicationEnabled, other.deduplicationEnabled) && Objects.equals(persistence, other.persistence) && Objects.equals(bundles, other.bundles) && Objects.equals(latency_stats_sample_rate, other.latency_stats_sample_rate) @@ -108,6 +110,7 @@ public String toString() { .add("backlog_quota_map", backlog_quota_map).add("persistence", persistence) .add("deduplicationEnabled", deduplicationEnabled) .add("clusterDispatchRate", clusterDispatchRate) + .add("clusterSubscribeRate", clusterSubscribeRate) .add("latency_stats_sample_rate", latency_stats_sample_rate) .add("antiAffinityGroup", antiAffinityGroup) .add("message_ttl_in_seconds", message_ttl_in_seconds).add("retention_policies", retention_policies) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscribeRate.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscribeRate.java new file mode 100644 index 0000000000..54fa2c7f0e --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscribeRate.java @@ -0,0 +1,59 @@ +/** + * 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.common.policies.data; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; + +public class SubscribeRate { + + public int subscribeThrottlingRatePerConsumer = -1; + public int ratePeriodInSecond = 30; + + public SubscribeRate() { + } + + public SubscribeRate(int subscribeThrottlingRatePerConsumer, int ratePeriodInSecond) { + this.subscribeThrottlingRatePerConsumer = subscribeThrottlingRatePerConsumer; + this.ratePeriodInSecond = ratePeriodInSecond; + } + + @Override + public int hashCode() { + return Objects.hash(subscribeThrottlingRatePerConsumer, ratePeriodInSecond); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof SubscribeRate) { + SubscribeRate rate = (SubscribeRate) obj; + return Objects.equals(subscribeThrottlingRatePerConsumer, rate.subscribeThrottlingRatePerConsumer) + && Objects.equals(ratePeriodInSecond, rate.ratePeriodInSecond); + } + return false; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("subscribeThrottlingRatePerConsumer", subscribeThrottlingRatePerConsumer) + .add("ratePeriodInSecond", ratePeriodInSecond).toString(); + } +} ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services