From commits-return-22408-archive-asf-public=cust-asf.ponee.io@pulsar.apache.org Wed Feb 13 18:47:59 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 735D9180674 for ; Wed, 13 Feb 2019 19:47:59 +0100 (CET) Received: (qmail 20406 invoked by uid 500); 13 Feb 2019 18:47:58 -0000 Mailing-List: contact commits-help@pulsar.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@pulsar.apache.org Delivered-To: mailing list commits@pulsar.apache.org Received: (qmail 20381 invoked by uid 99); 13 Feb 2019 18:47:58 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 13 Feb 2019 18:47:58 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id 060AF82DDF; Wed, 13 Feb 2019 18:47:58 +0000 (UTC) Date: Wed, 13 Feb 2019 18:47:57 +0000 To: "commits@pulsar.apache.org" Subject: [pulsar] branch master updated: [pulsar-broker] provide option to disable redelivery-tracker to reduce in memory positionImpl footprint and gc improvement (#3584) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <155008367778.25266.18036599031311930735@gitbox.apache.org> From: rdhabalia@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: pulsar X-Git-Refname: refs/heads/master X-Git-Reftype: branch X-Git-Oldrev: 200d175a02638086c850c732b6bd0f285aff7f42 X-Git-Newrev: 14a596288c6d372236b46b418b8df25238410ad3 X-Git-Rev: 14a596288c6d372236b46b418b8df25238410ad3 X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated This is an automated email from the ASF dual-hosted git repository. rdhabalia pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/pulsar.git The following commit(s) were added to refs/heads/master by this push: new 14a5962 [pulsar-broker] provide option to disable redelivery-tracker to reduce in memory positionImpl footprint and gc improvement (#3584) 14a5962 is described below commit 14a596288c6d372236b46b418b8df25238410ad3 Author: Rajan Dhabalia AuthorDate: Wed Feb 13 10:47:52 2019 -0800 [pulsar-broker] provide option to disable redelivery-tracker to reduce in memory positionImpl footprint and gc improvement (#3584) * [pulsar-broker] provide option to disable redelivery-tracker to reduce in memory positionImpl footprint and gc improvement * change config name --- conf/broker.conf | 3 +++ conf/standalone.conf | 3 +++ .../main/java/org/apache/pulsar/broker/ServiceConfiguration.java | 7 +++++++ .../service/persistent/PersistentDispatcherMultipleConsumers.java | 7 +++++-- site2/docs/reference-configuration.md | 1 + 5 files changed, 19 insertions(+), 2 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index a80f808..befb9ba 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -95,6 +95,9 @@ activeConsumerFailoverDelayTimeMillis=1000 # When it is 0, inactive subscriptions are not deleted automatically subscriptionExpirationTimeMinutes=0 +# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled) +subscriptionRedeliveryTrackerEnabled=true + # How frequently to proactively check and purge expired subscription subscriptionExpiryCheckIntervalInMinutes=5 diff --git a/conf/standalone.conf b/conf/standalone.conf index 96a906f..a8e7181 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -76,6 +76,9 @@ activeConsumerFailoverDelayTimeMillis=1000 # When it is 0, inactive subscriptions are not deleted automatically subscriptionExpirationTimeMinutes=0 +# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled) +subscriptionRedeliveryTrackerEnabled=true + # How frequently to proactively check and purge expired subscription subscriptionExpiryCheckIntervalInMinutes=5 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 24f3e32..5d60f72 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 @@ -237,6 +237,13 @@ public class ServiceConfiguration implements PulsarConfiguration { ) private long subscriptionExpirationTimeMinutes = 0; @FieldContext( + category = CATEGORY_POLICIES, + dynamic = true, + doc = "Enable subscription message redelivery tracker to send redelivery " + + "count to consumer (default is enabled)" + ) + private boolean subscriptionRedeliveryTrackerEnabled = true; + @FieldContext( category = CATEGORY_POLICIES, doc = "How frequently to proactively check and purge expired subscription" ) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 0586544..e1ff3ea 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -46,6 +46,7 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Consumer.SendMessageInfo; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; @@ -97,15 +98,17 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMu } public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor) { + this.serviceConfig = topic.getBrokerService().pulsar().getConfiguration(); this.cursor = cursor; this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); this.topic = topic; this.messagesToReplay = new ConcurrentLongPairSet(512, 2); - this.redeliveryTracker = new InMemoryRedeliveryTracker(); + this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() + ? new InMemoryRedeliveryTracker() + : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; this.readBatchSize = MaxReadBatchSize; this.maxUnackedMessages = topic.getBrokerService().pulsar().getConfiguration() .getMaxUnackedMessagesPerSubscription(); - this.serviceConfig = topic.getBrokerService().pulsar().getConfiguration(); this.initializeDispatchRateLimiterIfNeeded(Optional.empty()); } diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md index 753a8fb..08354bd 100644 --- a/site2/docs/reference-configuration.md +++ b/site2/docs/reference-configuration.md @@ -146,6 +146,7 @@ Pulsar brokers are responsible for handling incoming messages from producers, di |tokenPublicKey| Configure the public key to be used to validate auth tokens. The key can be specified like: `tokenPublicKey=data:base64,xxxxxxxxx` or `tokenPublicKey=file:///my/secret.key`|| |maxUnackedMessagesPerConsumer| Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending messages to consumer once, this limit reaches until consumer starts acknowledging messages back. Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction |50000| |maxUnackedMessagesPerSubscription| Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit check and dispatcher can dispatch messages without any restriction |200000| +|subscriptionRedeliveryTrackerEnabled| Enable subscription message redelivery tracker |true| |maxConcurrentLookupRequest| Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic |50000| |maxConcurrentTopicLoadRequest| Max number of concurrent topic loading request broker allows to control number of zk-operations |5000| |authenticationEnabled| Enable authentication |false|