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

Reply via email to