├── .gitignore ├── LICENSE ├── README.md ├── consumer-java ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── pulsar │ └── client │ └── impl │ ├── ReferenceMultiTopicsConsumerImpl.java │ ├── WeightedConsumerBuilder.java │ ├── WeightedMultiTopicsConsumerImpl.java │ └── weight │ ├── ExponentialWeightDistribution.java │ ├── LinearWeightDistribution.java │ ├── RangeDistribution.java │ ├── TopicMessageCountTracker.java │ ├── TopicThresholdDistribution.java │ ├── TopicThresholdDistributionImpl.java │ ├── WeightDistribution.java │ └── WeightedConsumerConfiguration.java ├── docs └── images │ ├── lagging_backlog.png │ ├── lagging_thpt.png │ └── tailing_thpt.png └── examples ├── pom.xml └── src └── main ├── java └── examples │ ├── StatPrinter.java │ └── UnorderedConsumptionJob.java └── resources └── log4j2.xml /.gitignore: -------------------------------------------------------------------------------- 1 | ### Maven template 2 | target/ 3 | pom.xml.tag 4 | pom.xml.releaseBackup 5 | pom.xml.versionsBackup 6 | pom.xml.next 7 | release.properties 8 | dependency-reduced-pom.xml 9 | buildNumber.properties 10 | .mvn/timing.properties 11 | # https://github.com/takari/maven-wrapper#usage-without-binary-jar 12 | .mvn/wrapper/maven-wrapper.jar 13 | 14 | ### JetBrains 15 | *.iml 16 | *.ipr 17 | *.iws 18 | /.idea/ 19 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Pulsar Weighted Consumer (PWC) 2 | 3 | This repository provides implementation of a pulsar consumer that can do priority consumption across multiple topics. Topics and associated weights are supplied during consumer creation. The unequal consumption is done on a best effort basis and is best observed when all the input topics have sufficient backlog. 4 | 5 | > Note: `WeightedMultiTopicsConsumerImpl` is a fork of the MultiTopicsConsumerImpl that ships with java pulsar-client. A neat extension of the same class is not possible at this moment since we need to change certain private members. 6 | 7 | ## Adding PWC to your build 8 | 9 | ### Dependency and Versioning 10 | [Pulsar java client](https://pulsar.apache.org/docs/client-libraries-java/) should be added as a dependency in the maven project where you intend to import PWC. PWC builds are published follows a versioning scheme aligned with Pulsar: `major.minor-X`, where major.minor should be the same as the included pulsar client. Since pulsar clients are fully compatible across versions, it should be possible to use a different pulsar-client as compared to PWC, but this is neither tested nor recommended. 11 | 12 | A good strategy would be to look at the PWC builds already available and align your pulsar-client to one of those versions. 13 | ```xml 14 | 15 | com.flipkart.viesti 16 | pulsar-weighted-consumer 17 | 2.7-1 18 | 19 | ``` 20 | 21 | If you have a hard requirement of using a specific version of pulsar-client, but PWC has not been published for that pulsar verion, feel free to open a ticket. 22 | 23 | ### Snapshots 24 | 25 | Snapshots of PWC are published using version as `major.minor-SNAPSHOT`. For example: `com.flipkart:pulsar-weighted-consumer:2.7-SNAPSHOT` 26 | 27 | ### Artifactory and Releases 28 | 29 | Maven/Gradle releases are published to [clojars artifactory here](https://clojars.org/com.flipkart/pulsar-weighted-consumer). You need to have clojars repository added in your build file. Example for pom: 30 | ```xml 31 | 32 | 33 | clojars 34 | Clojars repository 35 | https://clojars.org/repo 36 | 37 | 38 | ``` 39 | 40 | You can also find more information about [changelog and releases here](https://github.com/flipkart-incubator/pulsar-weighted-consumer/releases). 41 | 42 | TODO: javadocs link 43 | 44 | ## Learn about PWC 45 | 46 | ### Concepts 47 | 48 | Pulsar weighted consumer modifies the original multi topic consumer and introduces unequal consumptions of messages across different topics. Multi topic consumer of Pulsar works by creating multiple topic specific consumers and draining messages from each of them to a shared queue. This shared queue in turn is drained by the user using `receive` method variants. PWC introduces few changes like keeping count of messages in the shared queue at a topic granularity and assigning weighted thresholds to the topics. Note that here a threshold or count is cumulatively applied to all partitions of a topic. When the message count breaches the topic's threshold, fetch of messages from its internal sub-consumer is paused. To ensure adherence to weights, it remains paused unless all the messages from the shared queue have not been drained (this is configurable though). 49 | 50 | #### Bounds and Weights 51 | 52 | The thresholds are determined by `minBound` which is associated with the lowest weighted topic and `maxBound` associate with the highest weighted topic. The thresholds for rest of the topics are calculated using the `distributionStrategy` specified. Consider an example of `minBound=100, maxBound=1000, distributionStrategy=LINEAR` and 5 topics associated with weights 1 to 5, respectively. Here the thresholds for the topics get computed as: `[100, 325, 550, 775, 1000]` which is essentially a linear increment of 225 messages between consecutive weights. Users can tweak minBound, maxBound, distributionStrategy and weights to arrive at the relative thresholds for messages of different topics in the shared queue. More the messages of a given topic in the shared queue, higher the consumption throughput for that topic (not true everytime though, refer to force priority consumption section). 53 | 54 | > **NOTE** 55 | > If w_max and w_min are the highest and lowest weights that you assign to the topics and if `w_max - w_min < 10`, it is recommended to not change the minBound and maxBound defaults. 56 | 57 | #### Bursting 58 | 59 | Although the fetch of messages from subconsumer is paused, the subconsumer can continue to fetch messages independently over the network from the broker, as per its receiver queue size. Since network I/O is significantly costlier than the local subconsumer to main consumer fetch, the pause of an internal topic consumer does not introduce a measurable penalty. 60 | 61 | The above design becomes relevant when you have few high throughput topics of lesser priority (smaller threshold) in tandem with low throughput topics of higher priority. What we do not want is a behaviour where high throughput topics are artificially penalized and overall consumption rate to remain low despite the consumer having capacity to process more messages. PWC exhibits the behaviour of bursting where if high priority topics do not have enough messages available, overall consumption rate does not suffer and lower priority topics can burst. 62 | 63 | #### Force Priority Consumption 64 | 65 | A side effect of having support for bursting is that when you have enough messages to consume across all topics and message processing is near instantaneous on the local consumer (think of printing message or incrementing a counter and subsequently acking), then all the topics regardless of weights assigned will show a high and near-equal consumption rate. This may be the desired behaviour in majority of the situations where weights do no matter much if local message processing is much faster than the cost to fetch the messages in the first place over the network. 66 | 67 | But if in situations like above, strict priority consumption is required, then a separate flag `throttleReceieverQueue` can be enabled which applies weighted thresholds even to the messages fetched by specific topic consumers from the brokers. Usage of this flag should only be done in very specific scenarios, and you are almost always better off leaving this untouched because once you turn this on, it disables bursting. 68 | 69 | ### Example Usage 70 | 71 | Instead of using the default consumer builder like `pulsarClient.newConsumer()...`, you would instantiate an instance of WeightedConsumerBuilder, configure all the weight related flags and then configure rest of the subscription specific properties. 72 | 73 | ```java 74 | Map topicWeights = new HashMap() {{ 75 | put("persistent://abc-tenant/xyz-ns/topic1", 1); 76 | put("persistent://abc-tenant/xyz-ns/topic2", 2); 77 | put("persistent://abc-tenant/xyz-ns/topic3", 3); 78 | }}; 79 | 80 | WeightedConsumerBuilder consumerBuilder = new WeightedConsumerBuilder<>(client, Schema.BYTES) 81 | .distributionStrategy(WeightedConsumerConfiguration.DistributionStrategy.EXPONENTIAL) 82 | .topics(topicWeights); 83 | 84 | Consumer consumer = consumerBuilder 85 | .subscriptionName("sub-a") 86 | .subscriptionType(SubscriptionType.Shared) //all subscription types are supported 87 | .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) 88 | .subscribe(); 89 | ``` 90 | 91 | The consumer behaviour is 100% backwards compatible and all the subscription or client related configuration flags are available. Any topic that is added through older overloads like `topics(String...)` defaults to weight=1. Use of the newer overload `topics(Map)` is strongly recommended. 92 | 93 | ### Configuration 94 | 95 | Different weight related flags have been introduced which can be configured through WeightedConsumerBuilder. 96 | 97 | | Method | Default | Description | 98 | |-------------------------------|---------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| 99 | | topics(Map) | | Map of topics and associated weights. Weight has to be in the range [1,maxWeightAllowed].
Topic names can follow a suffix convention with weight included in the names, like so: `persistent://public/default/hello-world-weight-2`. Here the weight is parsed as 2.
Precedence order: weight override supplied in the map > weight specified in the name > 1 (if weight cannot be determined) | 100 | | maxWeightAllowed(int) | 100 | Default should be good enough in most of the circumstances | 101 | | minBound(int) | 100 | Minimum number of messages to be fetched for a topic in the shared queue. Applies to the lowest weighted topic
Default should be good enough in most of the circumstances | 102 | | maxBound(int) | 1000 | Maximum number of messages to be fetched for a topic in the shared queue. Applies to the highest weighted topic.
Also used to set the receiver queue size for all internal consumers. Existing receiver queue size property of subscription is ignored.
Default should be good enough in most of the circumstances | 103 | | distributionStrategy(enum) | LINEAR | Determines how bounds/thresholds are distributed between lowest weighted and highest weighted topic.
Available values: LINEAR, EXPONENTIAL | 104 | | retryTopicWeight(int) | 1 | Assign a weight to the retry topic as configured in the retry topic policies of the consumer | 105 | | throttleReceiveQueue(boolean) | false | Do not switch to `true` unless you understand why. Refer to Concepts#Force Priority Consumption to understand this flag better.
Default is good enough for almost all circumstances. | 106 | | queueResumeThreshold(int) | 0 | Caution! Do not change this flag unless you understand why. Refer to Concepts to understand how shared queue and thresholds interact.
Default is good enough for almost all circumstances. | 107 | 108 | ## Test Results 109 | 110 | Topics and subscriptions were setup like so: 111 | * 3 topics (4 partitions each) with weights [1,2,3] respectively 112 | * Enough backlog of messages was present in the topics for testing lagging consumers. Tailing consumer tests were done with zero backlog. 113 | * 1 "shared" subscription per topic using PWC with minBound=100, maxBound=1000, distribution=LINEAR 114 | 115 | ### Lagging consumers 116 | 117 | * Messages were processed in parallel across 4 worker threads. 118 | * Message processing time was simulated to be 5ms. 119 | * Unequal consumption rate across subs of different topics was observed. 120 | * Subsequently, backlog was observed to reduce at different rates for each sub. 121 | 122 | ![](docs/images/lagging_thpt.png) 123 | ![](docs/images/lagging_backlog.png) 124 | 125 | ### Tailing consumers 126 | 127 | * Messages were processed in parallel across 4 worker threads. 128 | * Message processing was near instantaneous (no simulated delay) 129 | * All subs are able to keep up with the produce rate. This demonstrates bursting in consumption rate of lower weighted topics when message processing can keep up. 130 | 131 | ![](docs/images/tailing_thpt.png) 132 | 133 | ## How to develop? 134 | 135 | ### Branching and Release Strategy 136 | 137 | * The repository follows branching strategy as `branch-major.minor` where major.minor aligns with the pulsar versions. 138 | * `main` branch corresponds typically with the latest pulsar release and used for development purposes. Releases are done from specific branches. 139 | * Bug fixes specific to a pulsar version can be done in the relevant branch. 140 | * Patches which apply to all pulsar versions can be merged directly to `main` and cherry-picked to other branches for release. 141 | 142 | ### How to do code contributions? 143 | 144 | Start a PR from your fork with the base branch set correctly. Base branch should be master if this is a new feature or a bug fix that affects the latest version. Base branch can be specific pulsar version branch like `branch-2.7` if this a hotfix for a specific release. PR should cover the following aspects: 145 | 146 | * Clarifies the purpose and the need of this change. In case of a bug, ensure you have clearly documented the affected library version, jdk version, stack traces, logs and reproduction steps (if applicable). 147 | * In case of a non-trivial design change, the PR description should explain that succintly and attach test results for the key test cases (mentioned elsewhere in the doc) to ensure there has been no regression. 148 | * Inline comments for non obvious behaviour or in case of a hacky workaround as well as README doc changes should be included in the PR is applicable. 149 | * Backwards compatibility of client facing behaviour should be adhered to. 150 | -------------------------------------------------------------------------------- /consumer-java/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.flipkart 8 | pulsar-weighted-consumer 9 | 2.10-0 10 | 11 | 12 | 8 13 | 2.10.0 14 | 15 | 16 | 17 | 18 | The Apache Software License, Version 2.0 19 | http://www.apache.org/licenses/LICENSE-2.0.txt 20 | 21 | 22 | 23 | 24 | 25 | clojars 26 | Clojars repository 27 | https://clojars.org/repo 28 | 29 | 30 | 31 | 32 | 33 | clojars 34 | Clojars repository 35 | https://clojars.org/repo 36 | 37 | 38 | central 39 | Maven Central 40 | https://repo1.maven.org/maven2/ 41 | 42 | 43 | 44 | 45 | 46 | 47 | org.apache.maven.plugins 48 | maven-compiler-plugin 49 | 50 | ${java.version} 51 | ${java.version} 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | org.apache.pulsar 60 | pulsar-client 61 | ${pulsar.version} 62 | provided 63 | 64 | 65 | 66 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/ReferenceMultiTopicsConsumerImpl.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | *

10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | *

12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.pulsar.client.impl; 20 | 21 | import org.apache.pulsar.client.api.*; 22 | import org.apache.pulsar.client.api.PulsarClientException.NotSupportedException; 23 | import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; 24 | import org.apache.pulsar.client.impl.transaction.TransactionImpl; 25 | import org.apache.pulsar.client.util.ConsumerName; 26 | import org.apache.pulsar.client.util.ExecutorProvider; 27 | import org.apache.pulsar.common.api.proto.CommandAck.AckType; 28 | import org.apache.pulsar.common.naming.TopicName; 29 | import org.apache.pulsar.common.partition.PartitionedTopicMetadata; 30 | import org.apache.pulsar.common.util.CompletableFutureCancellationHandler; 31 | import org.apache.pulsar.common.util.FutureUtil; 32 | import org.apache.pulsar.shade.com.google.common.annotations.VisibleForTesting; 33 | import org.apache.pulsar.shade.com.google.common.collect.ImmutableMap; 34 | import org.apache.pulsar.shade.com.google.common.collect.Lists; 35 | import org.apache.pulsar.shade.io.netty.util.Timeout; 36 | import org.apache.pulsar.shade.io.netty.util.TimerTask; 37 | import org.apache.pulsar.shade.org.apache.commons.lang3.tuple.Pair; 38 | import org.slf4j.Logger; 39 | import org.slf4j.LoggerFactory; 40 | 41 | import java.util.*; 42 | import java.util.concurrent.*; 43 | import java.util.concurrent.atomic.AtomicBoolean; 44 | import java.util.concurrent.atomic.AtomicInteger; 45 | import java.util.concurrent.locks.ReadWriteLock; 46 | import java.util.concurrent.locks.ReentrantReadWriteLock; 47 | import java.util.function.Function; 48 | import java.util.stream.Collectors; 49 | import java.util.stream.IntStream; 50 | 51 | import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkArgument; 52 | import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkState; 53 | 54 | /** 55 | * Fork of {@link MultiTopicsConsumerImpl} from commit: b40e6eb7712821a8456880ca58de673bbc4bac36 56 | * @param 57 | */ 58 | class ReferenceMultiTopicsConsumerImpl extends ConsumerBase { 59 | 60 | public static final String DUMMY_TOPIC_NAME_PREFIX = "MultiTopicsConsumer-"; 61 | private static final Logger log = LoggerFactory.getLogger(ReferenceMultiTopicsConsumerImpl.class); 62 | // Map , store partition number for each topic 63 | protected final ConcurrentHashMap partitionedTopics; 64 | // Map , when get do ACK, consumer will by find by topic name 65 | private final ConcurrentHashMap> consumers; 66 | // Queue of partition consumers on which we have stopped calling receiveAsync() because the 67 | // shared incoming queue was full 68 | private final ConcurrentLinkedQueue> pausedConsumers; 69 | // Threshold for the shared queue. When the size of the shared queue goes below the threshold, we are going to 70 | // resume receiving from the paused consumer partitions 71 | private final int sharedQueueResumeThreshold; 72 | private final Object pauseMutex = new Object(); 73 | private final ReadWriteLock lock = new ReentrantReadWriteLock(); 74 | private final ConsumerStatsRecorder stats; 75 | private final UnAckedMessageTracker unAckedMessageTracker; 76 | private final ConsumerConfigurationData internalConfig; 77 | private final long startMessageRollbackDurationInSec; 78 | // sum of topicPartitions, simple topic has 1, partitioned topic equals to partition number. 79 | AtomicInteger allTopicPartitionsNumber; 80 | TopicsPartitionChangedListener topicsPartitionChangedListener; 81 | CompletableFuture partitionsAutoUpdateFuture = null; 82 | private boolean paused = false; 83 | // timeout related to auto check and subscribe partition increasement 84 | private volatile Timeout partitionsAutoUpdateTimeout = null; 85 | private volatile BatchMessageIdImpl startMessageId = null; 86 | 87 | ReferenceMultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, 88 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 89 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { 90 | this(client, DUMMY_TOPIC_NAME_PREFIX + ConsumerName.generateRandomName(), conf, executorProvider, 91 | subscribeFuture, schema, interceptors, createTopicIfDoesNotExist); 92 | } 93 | 94 | ReferenceMultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, 95 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 96 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, 97 | long startMessageRollbackDurationInSec) { 98 | this(client, DUMMY_TOPIC_NAME_PREFIX + ConsumerName.generateRandomName(), conf, executorProvider, 99 | subscribeFuture, schema, interceptors, createTopicIfDoesNotExist, startMessageId, 100 | startMessageRollbackDurationInSec); 101 | } 102 | 103 | ReferenceMultiTopicsConsumerImpl(PulsarClientImpl client, String singleTopic, ConsumerConfigurationData conf, 104 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 105 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { 106 | this(client, singleTopic, conf, executorProvider, subscribeFuture, schema, interceptors, 107 | createTopicIfDoesNotExist, null, 0); 108 | } 109 | 110 | ReferenceMultiTopicsConsumerImpl(PulsarClientImpl client, String singleTopic, ConsumerConfigurationData conf, 111 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 112 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, 113 | long startMessageRollbackDurationInSec) { 114 | super(client, singleTopic, conf, Math.max(2, conf.getReceiverQueueSize()), executorProvider, subscribeFuture, 115 | schema, interceptors); 116 | 117 | checkArgument(conf.getReceiverQueueSize() > 0, 118 | "Receiver queue size needs to be greater than 0 for Topics Consumer"); 119 | 120 | this.partitionedTopics = new ConcurrentHashMap<>(); 121 | this.consumers = new ConcurrentHashMap<>(); 122 | this.pausedConsumers = new ConcurrentLinkedQueue<>(); 123 | this.sharedQueueResumeThreshold = maxReceiverQueueSize / 2; 124 | this.allTopicPartitionsNumber = new AtomicInteger(0); 125 | this.startMessageId = startMessageId != null ? new BatchMessageIdImpl(MessageIdImpl.convertToMessageIdImpl(startMessageId)) : null; 126 | this.startMessageRollbackDurationInSec = startMessageRollbackDurationInSec; 127 | 128 | if (conf.getAckTimeoutMillis() != 0) { 129 | if (conf.getTickDurationMillis() > 0) { 130 | this.unAckedMessageTracker = new UnAckedTopicMessageTracker(client, this, conf); 131 | } else { 132 | this.unAckedMessageTracker = new UnAckedTopicMessageTracker(client, this, conf); 133 | } 134 | } else { 135 | this.unAckedMessageTracker = UnAckedMessageTracker.UNACKED_MESSAGE_TRACKER_DISABLED; 136 | } 137 | 138 | this.internalConfig = getInternalConsumerConfig(); 139 | this.stats = client.getConfiguration().getStatsIntervalSeconds() > 0 ? new ConsumerStatsRecorderImpl(this) : null; 140 | 141 | // start track and auto subscribe partition increment 142 | if (conf.isAutoUpdatePartitions()) { 143 | topicsPartitionChangedListener = new TopicsPartitionChangedListener(); 144 | partitionsAutoUpdateTimeout = client.timer() 145 | .newTimeout(partitionsAutoUpdateTimerTask, conf.getAutoUpdatePartitionsIntervalSeconds(), TimeUnit.SECONDS); 146 | } 147 | 148 | if (conf.getTopicNames().isEmpty()) { 149 | setState(State.Ready); 150 | subscribeFuture().complete(ReferenceMultiTopicsConsumerImpl.this); 151 | return; 152 | } 153 | 154 | checkArgument(conf.getTopicNames().isEmpty() || topicNamesValid(conf.getTopicNames()), "Topics is empty or invalid."); 155 | 156 | List> futures = conf.getTopicNames().stream().map(t -> subscribeAsync(t, createTopicIfDoesNotExist)) 157 | .collect(Collectors.toList()); 158 | FutureUtil.waitForAll(futures) 159 | .thenAccept(finalFuture -> { 160 | if (allTopicPartitionsNumber.get() > maxReceiverQueueSize) { 161 | setMaxReceiverQueueSize(allTopicPartitionsNumber.get()); 162 | } 163 | setState(State.Ready); 164 | // We have successfully created N consumers, so we can start receiving messages now 165 | startReceivingMessages(new ArrayList<>(consumers.values())); 166 | log.info("[{}] [{}] Created topics consumer with {} sub-consumers", 167 | topic, subscription, allTopicPartitionsNumber.get()); 168 | subscribeFuture().complete(ReferenceMultiTopicsConsumerImpl.this); 169 | }) 170 | .exceptionally(ex -> { 171 | log.warn("[{}] Failed to subscribe topics: {}, closing consumer", topic, ex.getMessage()); 172 | closeAsync().whenComplete((res, closeEx) -> { 173 | if (closeEx != null) { 174 | log.error("[{}] Failed to unsubscribe after failed consumer creation: {}", topic, closeEx.getMessage()); 175 | } 176 | subscribeFuture.completeExceptionally(ex); 177 | }); 178 | return null; 179 | }); 180 | } 181 | 182 | // Check topics are valid. 183 | // - each topic is valid, 184 | // - topic names are unique. 185 | private static boolean topicNamesValid(Collection topics) { 186 | checkState(topics != null && topics.size() >= 1, 187 | "topics should contain more than 1 topic"); 188 | 189 | Optional result = topics.stream() 190 | .filter(topic -> !TopicName.isValid(topic)) 191 | .findFirst(); 192 | 193 | if (result.isPresent()) { 194 | log.warn("Received invalid topic name: {}", result.get()); 195 | return false; 196 | } 197 | 198 | // check topic names are unique 199 | HashSet set = new HashSet<>(topics); 200 | if (set.size() == topics.size()) { 201 | return true; 202 | } else { 203 | log.warn("Topic names not unique. unique/all : {}/{}", set.size(), topics.size()); 204 | return false; 205 | } 206 | } 207 | 208 | // create consumer for a single topic with already known partitions. 209 | // first create a consumer with no topic, then do subscription for already know partitionedTopic. 210 | public static ReferenceMultiTopicsConsumerImpl createPartitionedConsumer(PulsarClientImpl client, 211 | ConsumerConfigurationData conf, 212 | ExecutorProvider executorProvider, 213 | CompletableFuture> subscribeFuture, 214 | int numPartitions, 215 | Schema schema, ConsumerInterceptors interceptors) { 216 | checkArgument(conf.getTopicNames().size() == 1, "Should have only 1 topic for partitioned consumer"); 217 | 218 | // get topic name, then remove it from conf, so constructor will create a consumer with no topic. 219 | ConsumerConfigurationData cloneConf = conf.clone(); 220 | String topicName = cloneConf.getSingleTopic(); 221 | cloneConf.getTopicNames().remove(topicName); 222 | 223 | CompletableFuture future = new CompletableFuture<>(); 224 | ReferenceMultiTopicsConsumerImpl consumer = new ReferenceMultiTopicsConsumerImpl(client, topicName, cloneConf, executorProvider, 225 | future, schema, interceptors, true /* createTopicIfDoesNotExist */); 226 | 227 | future.thenCompose(c -> ((ReferenceMultiTopicsConsumerImpl) c).subscribeAsync(topicName, numPartitions)) 228 | .thenRun(() -> subscribeFuture.complete(consumer)) 229 | .exceptionally(e -> { 230 | log.warn("Failed subscription for createPartitionedConsumer: {} {}, e:{}", 231 | topicName, numPartitions, e); 232 | consumer.cleanupMultiConsumer(); 233 | subscribeFuture.completeExceptionally( 234 | PulsarClientException.wrap(((Throwable) e).getCause(), String.format("Failed to subscribe %s with %d partitions", topicName, numPartitions))); 235 | return null; 236 | }); 237 | return consumer; 238 | } 239 | 240 | public static boolean isIllegalMultiTopicsMessageId(MessageId messageId) { 241 | //only support earliest/latest 242 | return !MessageId.earliest.equals(messageId) && !MessageId.latest.equals(messageId); 243 | } 244 | 245 | private void startReceivingMessages(List> newConsumers) { 246 | if (log.isDebugEnabled()) { 247 | log.debug("[{}] startReceivingMessages for {} new consumers in topics consumer, state: {}", 248 | topic, newConsumers.size(), getState()); 249 | } 250 | 251 | if (getState() == State.Ready) { 252 | newConsumers.forEach(consumer -> { 253 | consumer.increaseAvailablePermits(consumer.getConnectionHandler().cnx(), conf.getReceiverQueueSize()); 254 | internalPinnedExecutor.execute(() -> receiveMessageFromConsumer(consumer)); 255 | }); 256 | } 257 | } 258 | 259 | private void receiveMessageFromConsumer(ConsumerImpl consumer) { 260 | consumer.receiveAsync().thenAccept(message -> { 261 | if (log.isDebugEnabled()) { 262 | log.debug("[{}] [{}] Receive message from sub consumer:{}", 263 | topic, subscription, consumer.getTopic()); 264 | } 265 | // Process the message, add to the queue and trigger listener or async callback 266 | messageReceived(consumer, message); 267 | 268 | int size = incomingMessages.size(); 269 | if (size >= maxReceiverQueueSize 270 | || (size > sharedQueueResumeThreshold && !pausedConsumers.isEmpty())) { 271 | // mark this consumer to be resumed later: if No more space left in shared queue, 272 | // or if any consumer is already paused (to create fair chance for already paused consumers) 273 | pausedConsumers.add(consumer); 274 | 275 | // Since we din't get a mutex, the condition on the incoming queue might have changed after 276 | // we have paused the current consumer. We need to re-check in order to avoid this consumer 277 | // from getting stalled. 278 | resumeReceivingFromPausedConsumersIfNeeded(); 279 | } else { 280 | // Schedule next receiveAsync() if the incoming queue is not full. Use a different thread to avoid 281 | // recursion and stack overflow 282 | internalPinnedExecutor.execute(() -> receiveMessageFromConsumer(consumer)); 283 | } 284 | }); 285 | } 286 | 287 | private void messageReceived(ConsumerImpl consumer, Message message) { 288 | checkArgument(message instanceof MessageImpl); 289 | TopicMessageImpl topicMessage = new TopicMessageImpl<>(consumer.getTopic(), 290 | consumer.getTopicNameWithoutPartition(), message, consumer); 291 | 292 | if (log.isDebugEnabled()) { 293 | log.debug("[{}][{}] Received message from topics-consumer {}", 294 | topic, subscription, message.getMessageId()); 295 | } 296 | 297 | // if asyncReceive is waiting : return message to callback without adding to incomingMessages queue 298 | CompletableFuture> receivedFuture = nextPendingReceive(); 299 | if (receivedFuture != null) { 300 | unAckedMessageTracker.add(topicMessage.getMessageId()); 301 | completePendingReceive(receivedFuture, topicMessage); 302 | } else if (enqueueMessageAndCheckBatchReceive(topicMessage) && hasPendingBatchReceive()) { 303 | notifyPendingBatchReceivedCallBack(); 304 | } 305 | 306 | if (listener != null) { 307 | triggerListener(); 308 | } 309 | } 310 | 311 | @Override 312 | protected synchronized void messageProcessed(Message msg) { 313 | unAckedMessageTracker.add(msg.getMessageId()); 314 | decreaseIncomingMessageSize(msg); 315 | } 316 | 317 | private void resumeReceivingFromPausedConsumersIfNeeded() { 318 | if (incomingMessages.size() <= sharedQueueResumeThreshold && !pausedConsumers.isEmpty()) { 319 | while (true) { 320 | ConsumerImpl consumer = pausedConsumers.poll(); 321 | if (consumer == null) { 322 | break; 323 | } 324 | 325 | internalPinnedExecutor.execute(() -> { 326 | receiveMessageFromConsumer(consumer); 327 | }); 328 | } 329 | } 330 | } 331 | 332 | @Override 333 | protected Message internalReceive() throws PulsarClientException { 334 | Message message; 335 | try { 336 | message = incomingMessages.take(); 337 | decreaseIncomingMessageSize(message); 338 | checkState(message instanceof TopicMessageImpl); 339 | unAckedMessageTracker.add(message.getMessageId()); 340 | resumeReceivingFromPausedConsumersIfNeeded(); 341 | return message; 342 | } catch (Exception e) { 343 | throw PulsarClientException.unwrap(e); 344 | } 345 | } 346 | 347 | @Override 348 | protected Message internalReceive(int timeout, TimeUnit unit) throws PulsarClientException { 349 | Message message; 350 | try { 351 | message = incomingMessages.poll(timeout, unit); 352 | if (message != null) { 353 | decreaseIncomingMessageSize(message); 354 | checkArgument(message instanceof TopicMessageImpl); 355 | unAckedMessageTracker.add(message.getMessageId()); 356 | } 357 | resumeReceivingFromPausedConsumersIfNeeded(); 358 | return message; 359 | } catch (Exception e) { 360 | throw PulsarClientException.unwrap(e); 361 | } 362 | } 363 | 364 | @Override 365 | protected Messages internalBatchReceive() throws PulsarClientException { 366 | try { 367 | return internalBatchReceiveAsync().get(); 368 | } catch (InterruptedException | ExecutionException e) { 369 | State state = getState(); 370 | if (state != State.Closing && state != State.Closed) { 371 | stats.incrementNumBatchReceiveFailed(); 372 | throw PulsarClientException.unwrap(e); 373 | } else { 374 | return null; 375 | } 376 | } 377 | } 378 | 379 | @Override 380 | protected CompletableFuture> internalBatchReceiveAsync() { 381 | CompletableFutureCancellationHandler cancellationHandler = new CompletableFutureCancellationHandler(); 382 | CompletableFuture> result = cancellationHandler.createFuture(); 383 | try { 384 | lock.writeLock().lock(); 385 | if (hasEnoughMessagesForBatchReceive()) { 386 | MessagesImpl messages = getNewMessagesImpl(); 387 | Message msgPeeked = incomingMessages.peek(); 388 | while (msgPeeked != null && messages.canAdd(msgPeeked)) { 389 | Message msg = incomingMessages.poll(); 390 | if (msg != null) { 391 | decreaseIncomingMessageSize(msg); 392 | Message interceptMsg = beforeConsume(msg); 393 | messages.add(interceptMsg); 394 | } 395 | msgPeeked = incomingMessages.peek(); 396 | } 397 | result.complete(messages); 398 | } else { 399 | OpBatchReceive opBatchReceive = OpBatchReceive.of(result); 400 | pendingBatchReceives.add(opBatchReceive); 401 | cancellationHandler.setCancelAction(() -> pendingBatchReceives.remove(opBatchReceive)); 402 | } 403 | resumeReceivingFromPausedConsumersIfNeeded(); 404 | } finally { 405 | lock.writeLock().unlock(); 406 | } 407 | return result; 408 | } 409 | 410 | @Override 411 | protected CompletableFuture> internalReceiveAsync() { 412 | CompletableFutureCancellationHandler cancellationHandler = new CompletableFutureCancellationHandler(); 413 | CompletableFuture> result = cancellationHandler.createFuture(); 414 | Message message = incomingMessages.poll(); 415 | if (message == null) { 416 | pendingReceives.add(result); 417 | cancellationHandler.setCancelAction(() -> pendingReceives.remove(result)); 418 | } else { 419 | decreaseIncomingMessageSize(message); 420 | checkState(message instanceof TopicMessageImpl); 421 | unAckedMessageTracker.add(message.getMessageId()); 422 | resumeReceivingFromPausedConsumersIfNeeded(); 423 | result.complete(message); 424 | } 425 | return result; 426 | } 427 | 428 | @Override 429 | protected CompletableFuture doAcknowledge(MessageId messageId, AckType ackType, 430 | Map properties, 431 | TransactionImpl txnImpl) { 432 | checkArgument(messageId instanceof TopicMessageIdImpl); 433 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 434 | 435 | if (getState() != State.Ready) { 436 | return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); 437 | } 438 | 439 | if (ackType == AckType.Cumulative) { 440 | Consumer individualConsumer = consumers.get(topicMessageId.getTopicPartitionName()); 441 | if (individualConsumer != null) { 442 | MessageId innerId = topicMessageId.getInnerMessageId(); 443 | return individualConsumer.acknowledgeCumulativeAsync(innerId); 444 | } else { 445 | return FutureUtil.failedFuture(new PulsarClientException.NotConnectedException()); 446 | } 447 | } else { 448 | ConsumerImpl consumer = consumers.get(topicMessageId.getTopicPartitionName()); 449 | 450 | MessageId innerId = topicMessageId.getInnerMessageId(); 451 | return consumer.doAcknowledgeWithTxn(innerId, ackType, properties, txnImpl) 452 | .thenRun(() -> 453 | unAckedMessageTracker.remove(topicMessageId)); 454 | } 455 | } 456 | 457 | @Override 458 | protected CompletableFuture doAcknowledge(List messageIdList, AckType ackType, Map properties, TransactionImpl txn) { 459 | List> resultFutures = new ArrayList<>(); 460 | if (ackType == AckType.Cumulative) { 461 | messageIdList.forEach(messageId -> resultFutures.add(doAcknowledge(messageId, ackType, properties, txn))); 462 | return CompletableFuture.allOf(resultFutures.toArray(new CompletableFuture[0])); 463 | } else { 464 | if (getState() != State.Ready) { 465 | return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); 466 | } 467 | Map> topicToMessageIdMap = new HashMap<>(); 468 | for (MessageId messageId : messageIdList) { 469 | if (!(messageId instanceof TopicMessageIdImpl)) { 470 | return FutureUtil.failedFuture(new IllegalArgumentException("messageId is not instance of TopicMessageIdImpl")); 471 | } 472 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 473 | topicToMessageIdMap.putIfAbsent(topicMessageId.getTopicPartitionName(), new ArrayList<>()); 474 | topicToMessageIdMap.get(topicMessageId.getTopicPartitionName()).add(topicMessageId.getInnerMessageId()); 475 | } 476 | topicToMessageIdMap.forEach((topicPartitionName, messageIds) -> { 477 | ConsumerImpl consumer = consumers.get(topicPartitionName); 478 | resultFutures.add(consumer.doAcknowledgeWithTxn(messageIds, ackType, properties, txn) 479 | .thenAccept((res) -> messageIdList.forEach(unAckedMessageTracker::remove))); 480 | }); 481 | return CompletableFuture.allOf(resultFutures.toArray(new CompletableFuture[0])); 482 | } 483 | } 484 | 485 | @Override 486 | protected CompletableFuture doReconsumeLater(Message message, AckType ackType, 487 | Map properties, 488 | long delayTime, 489 | TimeUnit unit) { 490 | MessageId messageId = message.getMessageId(); 491 | checkArgument(messageId instanceof TopicMessageIdImpl); 492 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 493 | if (getState() != State.Ready) { 494 | return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); 495 | } 496 | 497 | if (ackType == AckType.Cumulative) { 498 | Consumer individualConsumer = consumers.get(topicMessageId.getTopicPartitionName()); 499 | if (individualConsumer != null) { 500 | MessageId innerId = topicMessageId.getInnerMessageId(); 501 | return individualConsumer.reconsumeLaterCumulativeAsync(message, delayTime, unit); 502 | } else { 503 | return FutureUtil.failedFuture(new PulsarClientException.NotConnectedException()); 504 | } 505 | } else { 506 | ConsumerImpl consumer = consumers.get(topicMessageId.getTopicPartitionName()); 507 | MessageId innerId = topicMessageId.getInnerMessageId(); 508 | return consumer.doReconsumeLater(message, ackType, properties, delayTime, unit) 509 | .thenRun(() -> unAckedMessageTracker.remove(topicMessageId)); 510 | } 511 | } 512 | 513 | @Override 514 | public void negativeAcknowledge(MessageId messageId) { 515 | checkArgument(messageId instanceof TopicMessageIdImpl); 516 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 517 | 518 | ConsumerImpl consumer = consumers.get(topicMessageId.getTopicPartitionName()); 519 | consumer.negativeAcknowledge(topicMessageId.getInnerMessageId()); 520 | } 521 | 522 | @Override 523 | public CompletableFuture unsubscribeAsync() { 524 | if (getState() == State.Closing || getState() == State.Closed) { 525 | return FutureUtil.failedFuture( 526 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 527 | } 528 | setState(State.Closing); 529 | 530 | CompletableFuture unsubscribeFuture = new CompletableFuture<>(); 531 | List> futureList = consumers.values().stream() 532 | .map(c -> c.unsubscribeAsync()).collect(Collectors.toList()); 533 | 534 | FutureUtil.waitForAll(futureList) 535 | .thenCompose((r) -> { 536 | setState(State.Closed); 537 | cleanupMultiConsumer(); 538 | log.info("[{}] [{}] [{}] Unsubscribed Topics Consumer", 539 | topic, subscription, consumerName); 540 | // fail all pending-receive futures to notify application 541 | return failPendingReceive(); 542 | }) 543 | .whenComplete((r, ex) -> { 544 | if (ex == null) { 545 | unsubscribeFuture.complete(null); 546 | } else { 547 | setState(State.Failed); 548 | unsubscribeFuture.completeExceptionally(ex); 549 | log.error("[{}] [{}] [{}] Could not unsubscribe Topics Consumer", 550 | topic, subscription, consumerName, ex.getCause()); 551 | } 552 | }); 553 | 554 | return unsubscribeFuture; 555 | } 556 | 557 | @Override 558 | public CompletableFuture closeAsync() { 559 | if (getState() == State.Closing || getState() == State.Closed) { 560 | unAckedMessageTracker.close(); 561 | return CompletableFuture.completedFuture(null); 562 | } 563 | setState(State.Closing); 564 | 565 | if (partitionsAutoUpdateTimeout != null) { 566 | partitionsAutoUpdateTimeout.cancel(); 567 | partitionsAutoUpdateTimeout = null; 568 | } 569 | 570 | CompletableFuture closeFuture = new CompletableFuture<>(); 571 | List> futureList = consumers.values().stream() 572 | .map(c -> c.closeAsync()).collect(Collectors.toList()); 573 | 574 | FutureUtil.waitForAll(futureList) 575 | .thenCompose((r) -> { 576 | setState(State.Closed); 577 | cleanupMultiConsumer(); 578 | log.info("[{}] [{}] Closed Topics Consumer", topic, subscription); 579 | // fail all pending-receive futures to notify application 580 | return failPendingReceive(); 581 | }) 582 | .whenComplete((r, ex) -> { 583 | if (ex == null) { 584 | closeFuture.complete(null); 585 | } else { 586 | setState(State.Failed); 587 | closeFuture.completeExceptionally(ex); 588 | log.error("[{}] [{}] Could not close Topics Consumer", topic, subscription, 589 | ex.getCause()); 590 | } 591 | }); 592 | 593 | return closeFuture; 594 | } 595 | 596 | private void cleanupMultiConsumer() { 597 | unAckedMessageTracker.close(); 598 | client.cleanupConsumer(this); 599 | } 600 | 601 | @Override 602 | public boolean isConnected() { 603 | return consumers.values().stream().allMatch(consumer -> consumer.isConnected()); 604 | } 605 | 606 | @Override 607 | String getHandlerName() { 608 | return subscription; 609 | } 610 | 611 | private ConsumerConfigurationData getInternalConsumerConfig() { 612 | ConsumerConfigurationData internalConsumerConfig = conf.clone(); 613 | internalConsumerConfig.setSubscriptionName(subscription); 614 | internalConsumerConfig.setConsumerName(consumerName); 615 | internalConsumerConfig.setMessageListener(null); 616 | return internalConsumerConfig; 617 | } 618 | 619 | @Override 620 | public void redeliverUnacknowledgedMessages() { 621 | lock.writeLock().lock(); 622 | try { 623 | consumers.values().stream().forEach(consumer -> { 624 | consumer.redeliverUnacknowledgedMessages(); 625 | consumer.unAckedChunkedMessageIdSequenceMap.clear(); 626 | }); 627 | incomingMessages.clear(); 628 | resetIncomingMessageSize(); 629 | unAckedMessageTracker.clear(); 630 | } finally { 631 | lock.writeLock().unlock(); 632 | } 633 | resumeReceivingFromPausedConsumersIfNeeded(); 634 | } 635 | 636 | @Override 637 | public void redeliverUnacknowledgedMessages(Set messageIds) { 638 | if (messageIds.isEmpty()) { 639 | return; 640 | } 641 | 642 | checkArgument(messageIds.stream().findFirst().get() instanceof TopicMessageIdImpl); 643 | 644 | if (conf.getSubscriptionType() != SubscriptionType.Shared) { 645 | // We cannot redeliver single messages if subscription type is not Shared 646 | redeliverUnacknowledgedMessages(); 647 | return; 648 | } 649 | removeExpiredMessagesFromQueue(messageIds); 650 | messageIds.stream().map(messageId -> (TopicMessageIdImpl) messageId) 651 | .collect(Collectors.groupingBy(TopicMessageIdImpl::getTopicPartitionName, Collectors.toSet())) 652 | .forEach((topicName, messageIds1) -> 653 | consumers.get(topicName) 654 | .redeliverUnacknowledgedMessages(messageIds1.stream() 655 | .map(mid -> mid.getInnerMessageId()).collect(Collectors.toSet()))); 656 | resumeReceivingFromPausedConsumersIfNeeded(); 657 | } 658 | 659 | @Override 660 | protected void completeOpBatchReceive(OpBatchReceive op) { 661 | notifyPendingBatchReceivedCallBack(op); 662 | resumeReceivingFromPausedConsumersIfNeeded(); 663 | } 664 | 665 | @Override 666 | public void seek(MessageId messageId) throws PulsarClientException { 667 | try { 668 | seekAsync(messageId).get(); 669 | } catch (Exception e) { 670 | throw PulsarClientException.unwrap(e); 671 | } 672 | } 673 | 674 | @Override 675 | public void seek(long timestamp) throws PulsarClientException { 676 | try { 677 | seekAsync(timestamp).get(); 678 | } catch (Exception e) { 679 | throw PulsarClientException.unwrap(e); 680 | } 681 | } 682 | 683 | @Override 684 | public void seek(Function function) throws PulsarClientException { 685 | try { 686 | this.seekAsync(function).get(); 687 | } catch (Exception var3) { 688 | throw PulsarClientException.unwrap(var3); 689 | } 690 | } 691 | 692 | @Override 693 | public CompletableFuture seekAsync(Function function) { 694 | List> futures = new ArrayList(this.consumers.size()); 695 | this.consumers.values().forEach((consumer) -> { 696 | futures.add(consumer.seekAsync(function)); 697 | }); 698 | this.unAckedMessageTracker.clear(); 699 | this.incomingMessages.clear(); 700 | this.resetIncomingMessageSize(); 701 | return FutureUtil.waitForAll(futures); 702 | } 703 | 704 | @Override 705 | public CompletableFuture seekAsync(MessageId messageId) { 706 | CompletableFuture seekFuture = new CompletableFuture<>(); 707 | MessageIdImpl targetMessageId = MessageIdImpl.convertToMessageIdImpl(messageId); 708 | if (targetMessageId == null || isIllegalMultiTopicsMessageId(messageId)) { 709 | seekFuture.completeExceptionally( 710 | new PulsarClientException("Illegal messageId, messageId can only be earliest/latest")); 711 | return seekFuture; 712 | } 713 | List> futures = new ArrayList<>(consumers.size()); 714 | consumers.values().forEach(consumerImpl -> futures.add(consumerImpl.seekAsync(targetMessageId))); 715 | 716 | unAckedMessageTracker.clear(); 717 | incomingMessages.clear(); 718 | resetIncomingMessageSize(); 719 | 720 | FutureUtil.waitForAll(futures).whenComplete((result, exception) -> { 721 | if (exception != null) { 722 | seekFuture.completeExceptionally(exception); 723 | } else { 724 | seekFuture.complete(result); 725 | } 726 | }); 727 | return seekFuture; 728 | } 729 | 730 | @Override 731 | public CompletableFuture seekAsync(long timestamp) { 732 | List> futures = new ArrayList<>(consumers.size()); 733 | consumers.values().forEach(consumer -> futures.add(consumer.seekAsync(timestamp))); 734 | return FutureUtil.waitForAll(futures); 735 | } 736 | 737 | @Override 738 | public int getAvailablePermits() { 739 | return consumers.values().stream().mapToInt(ConsumerImpl::getAvailablePermits).sum(); 740 | } 741 | 742 | @Override 743 | public boolean hasReachedEndOfTopic() { 744 | return consumers.values().stream().allMatch(Consumer::hasReachedEndOfTopic); 745 | } 746 | 747 | public boolean hasMessageAvailable() throws PulsarClientException { 748 | try { 749 | return hasMessageAvailableAsync().get(); 750 | } catch (Exception e) { 751 | throw PulsarClientException.unwrap(e); 752 | } 753 | } 754 | 755 | public CompletableFuture hasMessageAvailableAsync() { 756 | List> futureList = new ArrayList<>(); 757 | final AtomicBoolean hasMessageAvailable = new AtomicBoolean(false); 758 | for (ConsumerImpl consumer : consumers.values()) { 759 | futureList.add(consumer.hasMessageAvailableAsync().thenAccept(isAvailable -> { 760 | if (isAvailable) { 761 | hasMessageAvailable.compareAndSet(false, true); 762 | } 763 | })); 764 | } 765 | CompletableFuture completableFuture = new CompletableFuture<>(); 766 | FutureUtil.waitForAll(futureList).whenComplete((result, exception) -> { 767 | if (exception != null) { 768 | completableFuture.completeExceptionally(exception); 769 | } else { 770 | completableFuture.complete(hasMessageAvailable.get()); 771 | } 772 | }); 773 | return completableFuture; 774 | } 775 | 776 | @Override 777 | public int numMessagesInQueue() { 778 | return incomingMessages.size() + consumers.values().stream().mapToInt(ConsumerImpl::numMessagesInQueue).sum(); 779 | } 780 | 781 | @Override 782 | public synchronized ConsumerStats getStats() { 783 | if (stats == null) { 784 | return null; 785 | } 786 | stats.reset(); 787 | 788 | consumers.values().stream().forEach(consumer -> stats.updateCumulativeStats(consumer.getStats())); 789 | return stats; 790 | } 791 | 792 | public UnAckedMessageTracker getUnAckedMessageTracker() { 793 | return unAckedMessageTracker; 794 | } 795 | 796 | private void removeExpiredMessagesFromQueue(Set messageIds) { 797 | Message peek = incomingMessages.peek(); 798 | if (peek != null) { 799 | if (!messageIds.contains(peek.getMessageId())) { 800 | // first message is not expired, then no message is expired in queue. 801 | return; 802 | } 803 | 804 | // try not to remove elements that are added while we remove 805 | Message message = incomingMessages.poll(); 806 | checkState(message instanceof TopicMessageImpl); 807 | while (message != null) { 808 | decreaseIncomingMessageSize(message); 809 | MessageId messageId = message.getMessageId(); 810 | if (!messageIds.contains(messageId)) { 811 | messageIds.add(messageId); 812 | break; 813 | } 814 | message = incomingMessages.poll(); 815 | } 816 | } 817 | } 818 | 819 | private TopicName getTopicName(String topic) { 820 | try { 821 | return TopicName.get(topic); 822 | } catch (Exception ignored) { 823 | return null; 824 | } 825 | } 826 | 827 | private String getFullTopicName(String topic) { 828 | TopicName topicName = getTopicName(topic); 829 | return (topicName != null) ? topicName.toString() : null; 830 | } 831 | 832 | private void removeTopic(String topic) { 833 | String fullTopicName = getFullTopicName(topic); 834 | if (fullTopicName != null) { 835 | partitionedTopics.remove(topic); 836 | } 837 | } 838 | 839 | // subscribe one more given topic 840 | public CompletableFuture subscribeAsync(String topicName, boolean createTopicIfDoesNotExist) { 841 | TopicName topicNameInstance = getTopicName(topicName); 842 | if (topicNameInstance == null) { 843 | return FutureUtil.failedFuture( 844 | new PulsarClientException.AlreadyClosedException("Topic name not valid")); 845 | } 846 | String fullTopicName = topicNameInstance.toString(); 847 | if (consumers.containsKey(fullTopicName) || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { 848 | return FutureUtil.failedFuture( 849 | new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); 850 | } 851 | 852 | if (getState() == State.Closing || getState() == State.Closed) { 853 | return FutureUtil.failedFuture( 854 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 855 | } 856 | 857 | CompletableFuture subscribeResult = new CompletableFuture<>(); 858 | 859 | client.getPartitionedTopicMetadata(topicName) 860 | .thenAccept(metadata -> subscribeTopicPartitions(subscribeResult, fullTopicName, metadata.partitions, 861 | createTopicIfDoesNotExist)) 862 | .exceptionally(ex1 -> { 863 | log.warn("[{}] Failed to get partitioned topic metadata: {}", fullTopicName, ex1.getMessage()); 864 | subscribeResult.completeExceptionally(ex1); 865 | return null; 866 | }); 867 | 868 | return subscribeResult; 869 | } 870 | 871 | // subscribe one more given topic, but already know the numberPartitions 872 | @VisibleForTesting 873 | CompletableFuture subscribeAsync(String topicName, int numberPartitions) { 874 | TopicName topicNameInstance = getTopicName(topicName); 875 | if (topicNameInstance == null) { 876 | return FutureUtil.failedFuture( 877 | new PulsarClientException.AlreadyClosedException("Topic name not valid")); 878 | } 879 | String fullTopicName = topicNameInstance.toString(); 880 | if (consumers.containsKey(fullTopicName) || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { 881 | return FutureUtil.failedFuture( 882 | new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); 883 | } 884 | 885 | if (getState() == State.Closing || getState() == State.Closed) { 886 | return FutureUtil.failedFuture( 887 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 888 | } 889 | 890 | CompletableFuture subscribeResult = new CompletableFuture<>(); 891 | subscribeTopicPartitions(subscribeResult, fullTopicName, numberPartitions, true /* createTopicIfDoesNotExist */); 892 | 893 | return subscribeResult; 894 | } 895 | 896 | private void subscribeTopicPartitions(CompletableFuture subscribeResult, String topicName, int numPartitions, 897 | boolean createIfDoesNotExist) { 898 | client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((schema, cause) -> { 899 | if (null == cause) { 900 | doSubscribeTopicPartitions(schema, subscribeResult, topicName, numPartitions, createIfDoesNotExist); 901 | } else { 902 | subscribeResult.completeExceptionally(cause); 903 | } 904 | }); 905 | } 906 | 907 | private void doSubscribeTopicPartitions(Schema schema, 908 | CompletableFuture subscribeResult, String topicName, int numPartitions, 909 | boolean createIfDoesNotExist) { 910 | if (log.isDebugEnabled()) { 911 | log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); 912 | } 913 | 914 | List>> futureList; 915 | if (numPartitions != PartitionedTopicMetadata.NON_PARTITIONED) { 916 | // Below condition is true if subscribeAsync() has been invoked second time with same 917 | // topicName before the first invocation had reached this point. 918 | boolean isTopicBeingSubscribedForInOtherThread = this.partitionedTopics.putIfAbsent(topicName, numPartitions) != null; 919 | if (isTopicBeingSubscribedForInOtherThread) { 920 | String errorMessage = String.format("[%s] Failed to subscribe for topic [%s] in topics consumer. " 921 | + "Topic is already being subscribed for in other thread.", topic, topicName); 922 | log.warn(errorMessage); 923 | subscribeResult.completeExceptionally(new PulsarClientException(errorMessage)); 924 | return; 925 | } 926 | allTopicPartitionsNumber.addAndGet(numPartitions); 927 | 928 | int receiverQueueSize = Math.min(conf.getReceiverQueueSize(), 929 | conf.getMaxTotalReceiverQueueSizeAcrossPartitions() / numPartitions); 930 | ConsumerConfigurationData configurationData = getInternalConsumerConfig(); 931 | configurationData.setReceiverQueueSize(receiverQueueSize); 932 | 933 | futureList = IntStream 934 | .range(0, numPartitions) 935 | .mapToObj( 936 | partitionIndex -> { 937 | String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); 938 | CompletableFuture> subFuture = new CompletableFuture<>(); 939 | ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl(client, partitionName, 940 | configurationData, client.externalExecutorProvider(), 941 | partitionIndex, true, subFuture, 942 | startMessageId, schema, interceptors, 943 | createIfDoesNotExist, startMessageRollbackDurationInSec); 944 | consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); 945 | return subFuture; 946 | }) 947 | .collect(Collectors.toList()); 948 | } else { 949 | allTopicPartitionsNumber.incrementAndGet(); 950 | 951 | CompletableFuture> subFuture = new CompletableFuture<>(); 952 | consumers.compute(topicName, (key, existingValue) -> { 953 | if (existingValue != null) { 954 | String errorMessage = String.format("[%s] Failed to subscribe for topic [%s] in topics consumer. " 955 | + "Topic is already being subscribed for in other thread.", topic, topicName); 956 | log.warn(errorMessage); 957 | subscribeResult.completeExceptionally(new PulsarClientException(errorMessage)); 958 | return existingValue; 959 | } else { 960 | ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl(client, topicName, internalConfig, 961 | client.externalExecutorProvider(), -1, 962 | true, subFuture, null, schema, interceptors, 963 | createIfDoesNotExist); 964 | 965 | synchronized (pauseMutex) { 966 | if (paused) { 967 | newConsumer.pause(); 968 | } 969 | } 970 | return newConsumer; 971 | } 972 | }); 973 | 974 | futureList = Collections.singletonList(subFuture); 975 | } 976 | 977 | FutureUtil.waitForAll(futureList) 978 | .thenAccept(finalFuture -> { 979 | if (allTopicPartitionsNumber.get() > maxReceiverQueueSize) { 980 | setMaxReceiverQueueSize(allTopicPartitionsNumber.get()); 981 | } 982 | 983 | // We have successfully created new consumers, so we can start receiving messages for them 984 | startReceivingMessages(consumers.values().stream() 985 | .filter(consumer1 -> { 986 | String consumerTopicName = consumer1.getTopic(); 987 | return TopicName.get(consumerTopicName).getPartitionedTopicName().equals( 988 | TopicName.get(topicName).getPartitionedTopicName()); 989 | }) 990 | .collect(Collectors.toList())); 991 | 992 | subscribeResult.complete(null); 993 | log.info("[{}] [{}] Success subscribe new topic {} in topics consumer, partitions: {}, allTopicPartitionsNumber: {}", 994 | topic, subscription, topicName, numPartitions, allTopicPartitionsNumber.get()); 995 | return; 996 | }) 997 | .exceptionally(ex -> { 998 | handleSubscribeOneTopicError(topicName, ex, subscribeResult); 999 | return null; 1000 | }); 1001 | } 1002 | 1003 | // handling failure during subscribe new topic, unsubscribe success created partitions 1004 | private void handleSubscribeOneTopicError(String topicName, Throwable error, CompletableFuture subscribeFuture) { 1005 | log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, error.getMessage()); 1006 | client.externalExecutorProvider().getExecutor().submit(() -> { 1007 | AtomicInteger toCloseNum = new AtomicInteger(0); 1008 | consumers.values().stream().filter(consumer1 -> { 1009 | String consumerTopicName = consumer1.getTopic(); 1010 | if (TopicName.get(consumerTopicName).getPartitionedTopicName().equals(TopicName.get(topicName).getPartitionedTopicName())) { 1011 | toCloseNum.incrementAndGet(); 1012 | return true; 1013 | } else { 1014 | return false; 1015 | } 1016 | }).collect(Collectors.toList()).forEach(consumer2 -> { 1017 | consumer2.closeAsync().whenComplete((r, ex) -> { 1018 | consumer2.subscribeFuture().completeExceptionally(error); 1019 | allTopicPartitionsNumber.decrementAndGet(); 1020 | consumers.remove(consumer2.getTopic()); 1021 | if (toCloseNum.decrementAndGet() == 0) { 1022 | log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer, subscribe error: {}", 1023 | topic, topicName, error.getMessage()); 1024 | removeTopic(topicName); 1025 | subscribeFuture.completeExceptionally(error); 1026 | } 1027 | return; 1028 | }); 1029 | }); 1030 | }); 1031 | } 1032 | 1033 | // un-subscribe a given topic 1034 | public CompletableFuture unsubscribeAsync(String topicName) { 1035 | checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); 1036 | 1037 | if (getState() == State.Closing || getState() == State.Closed) { 1038 | return FutureUtil.failedFuture( 1039 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 1040 | } 1041 | 1042 | if (partitionsAutoUpdateTimeout != null) { 1043 | partitionsAutoUpdateTimeout.cancel(); 1044 | partitionsAutoUpdateTimeout = null; 1045 | } 1046 | 1047 | CompletableFuture unsubscribeFuture = new CompletableFuture<>(); 1048 | String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); 1049 | 1050 | List> consumersToUnsub = consumers.values().stream() 1051 | .filter(consumer -> { 1052 | String consumerTopicName = consumer.getTopic(); 1053 | return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); 1054 | }).collect(Collectors.toList()); 1055 | 1056 | List> futureList = consumersToUnsub.stream() 1057 | .map(ConsumerImpl::unsubscribeAsync).collect(Collectors.toList()); 1058 | 1059 | FutureUtil.waitForAll(futureList) 1060 | .whenComplete((r, ex) -> { 1061 | if (ex == null) { 1062 | consumersToUnsub.forEach(consumer1 -> { 1063 | consumers.remove(consumer1.getTopic()); 1064 | pausedConsumers.remove(consumer1); 1065 | allTopicPartitionsNumber.decrementAndGet(); 1066 | }); 1067 | 1068 | removeTopic(topicName); 1069 | ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); 1070 | 1071 | unsubscribeFuture.complete(null); 1072 | log.info("[{}] [{}] [{}] Unsubscribed Topics Consumer, allTopicPartitionsNumber: {}", 1073 | topicName, subscription, consumerName, allTopicPartitionsNumber); 1074 | } else { 1075 | unsubscribeFuture.completeExceptionally(ex); 1076 | setState(State.Failed); 1077 | log.error("[{}] [{}] [{}] Could not unsubscribe Topics Consumer", 1078 | topicName, subscription, consumerName, ex.getCause()); 1079 | } 1080 | }); 1081 | 1082 | return unsubscribeFuture; 1083 | } 1084 | 1085 | // Remove a consumer for a topic 1086 | public CompletableFuture removeConsumerAsync(String topicName) { 1087 | checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); 1088 | 1089 | if (getState() == State.Closing || getState() == State.Closed) { 1090 | return FutureUtil.failedFuture( 1091 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 1092 | } 1093 | 1094 | CompletableFuture unsubscribeFuture = new CompletableFuture<>(); 1095 | String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); 1096 | 1097 | 1098 | List> consumersToClose = consumers.values().stream() 1099 | .filter(consumer -> { 1100 | String consumerTopicName = consumer.getTopic(); 1101 | return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); 1102 | }).collect(Collectors.toList()); 1103 | 1104 | List> futureList = consumersToClose.stream() 1105 | .map(ConsumerImpl::closeAsync).collect(Collectors.toList()); 1106 | 1107 | FutureUtil.waitForAll(futureList) 1108 | .whenComplete((r, ex) -> { 1109 | if (ex == null) { 1110 | consumersToClose.forEach(consumer1 -> { 1111 | consumers.remove(consumer1.getTopic()); 1112 | pausedConsumers.remove(consumer1); 1113 | allTopicPartitionsNumber.decrementAndGet(); 1114 | }); 1115 | 1116 | removeTopic(topicName); 1117 | ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); 1118 | 1119 | unsubscribeFuture.complete(null); 1120 | log.info("[{}] [{}] [{}] Removed Topics Consumer, allTopicPartitionsNumber: {}", 1121 | topicName, subscription, consumerName, allTopicPartitionsNumber); 1122 | } else { 1123 | unsubscribeFuture.completeExceptionally(ex); 1124 | setState(State.Failed); 1125 | log.error("[{}] [{}] [{}] Could not remove Topics Consumer", 1126 | topicName, subscription, consumerName, ex.getCause()); 1127 | } 1128 | }); 1129 | 1130 | return unsubscribeFuture; 1131 | } 1132 | 1133 | // get topics name 1134 | public List getPartitionedTopics() { 1135 | return partitionedTopics.keySet().stream().collect(Collectors.toList()); 1136 | } 1137 | 1138 | // get partitioned topics name 1139 | public List getPartitions() { 1140 | return consumers.keySet().stream().collect(Collectors.toList()); 1141 | } 1142 | 1143 | // get partitioned consumers 1144 | public List> getConsumers() { 1145 | return consumers.values().stream().collect(Collectors.toList()); 1146 | } 1147 | 1148 | // get all partitions that in the topics map 1149 | int getPartitionsOfTheTopicMap() { 1150 | return partitionedTopics.values().stream().mapToInt(Integer::intValue).sum(); 1151 | } 1152 | 1153 | @Override 1154 | public void pause() { 1155 | synchronized (pauseMutex) { 1156 | paused = true; 1157 | consumers.forEach((name, consumer) -> consumer.pause()); 1158 | } 1159 | } 1160 | 1161 | @Override 1162 | public void resume() { 1163 | synchronized (pauseMutex) { 1164 | paused = false; 1165 | consumers.forEach((name, consumer) -> consumer.resume()); 1166 | } 1167 | } 1168 | 1169 | @Override 1170 | public long getLastDisconnectedTimestamp() { 1171 | long lastDisconnectedTimestamp = 0; 1172 | Optional> c = consumers.values().stream().max(Comparator.comparingLong(ConsumerImpl::getLastDisconnectedTimestamp)); 1173 | if (c.isPresent()) { 1174 | lastDisconnectedTimestamp = c.get().getLastDisconnectedTimestamp(); 1175 | } 1176 | return lastDisconnectedTimestamp; 1177 | } 1178 | 1179 | // subscribe increased partitions for a given topic 1180 | private CompletableFuture subscribeIncreasedTopicPartitions(String topicName) { 1181 | int oldPartitionNumber = partitionedTopics.get(topicName); 1182 | 1183 | return client.getPartitionsForTopic(topicName).thenCompose(list -> { 1184 | int currentPartitionNumber = Long.valueOf(list.stream().filter(t -> TopicName.get(t).isPartitioned()).count()).intValue(); 1185 | 1186 | if (log.isDebugEnabled()) { 1187 | log.debug("[{}] partitions number. old: {}, new: {}", 1188 | topicName, oldPartitionNumber, currentPartitionNumber); 1189 | } 1190 | 1191 | if (oldPartitionNumber == currentPartitionNumber) { 1192 | // topic partition number not changed 1193 | return CompletableFuture.completedFuture(null); 1194 | } else if (currentPartitionNumber == PartitionedTopicMetadata.NON_PARTITIONED) { 1195 | // The topic was initially partitioned but then it was deleted. We keep it in the topics 1196 | partitionedTopics.put(topicName, 0); 1197 | 1198 | allTopicPartitionsNumber.addAndGet(-oldPartitionNumber); 1199 | List> futures = new ArrayList<>(); 1200 | for (Iterator>> it = consumers.entrySet().iterator(); it.hasNext(); ) { 1201 | Map.Entry> e = it.next(); 1202 | String partitionedTopicName = TopicName.get(e.getKey()).getPartitionedTopicName(); 1203 | 1204 | // Remove the consumers that belong to the deleted partitioned topic 1205 | if (partitionedTopicName.equals(topicName)) { 1206 | futures.add(e.getValue().closeAsync()); 1207 | consumers.remove(e.getKey()); 1208 | } 1209 | } 1210 | 1211 | return FutureUtil.waitForAll(futures); 1212 | } else if (oldPartitionNumber < currentPartitionNumber) { 1213 | allTopicPartitionsNumber.addAndGet(currentPartitionNumber - oldPartitionNumber); 1214 | partitionedTopics.put(topicName, currentPartitionNumber); 1215 | List newPartitions = list.subList(oldPartitionNumber, currentPartitionNumber); 1216 | // subscribe new added partitions 1217 | List>> futureList = newPartitions 1218 | .stream() 1219 | .map(partitionName -> { 1220 | int partitionIndex = TopicName.getPartitionIndex(partitionName); 1221 | CompletableFuture> subFuture = new CompletableFuture<>(); 1222 | ConsumerConfigurationData configurationData = getInternalConsumerConfig(); 1223 | ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl( 1224 | client, partitionName, configurationData, 1225 | client.externalExecutorProvider(), 1226 | partitionIndex, true, subFuture, null, schema, interceptors, 1227 | true /* createTopicIfDoesNotExist */); 1228 | synchronized (pauseMutex) { 1229 | if (paused) { 1230 | newConsumer.pause(); 1231 | } 1232 | consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); 1233 | } 1234 | if (log.isDebugEnabled()) { 1235 | log.debug("[{}] create consumer {} for partitionName: {}", 1236 | topicName, newConsumer.getTopic(), partitionName); 1237 | } 1238 | return subFuture; 1239 | }) 1240 | .collect(Collectors.toList()); 1241 | 1242 | // wait for all partitions subscribe future complete, then startReceivingMessages 1243 | return FutureUtil.waitForAll(futureList) 1244 | .thenAccept(finalFuture -> { 1245 | List> newConsumerList = newPartitions.stream() 1246 | .map(partitionTopic -> consumers.get(partitionTopic)) 1247 | .collect(Collectors.toList()); 1248 | startReceivingMessages(newConsumerList); 1249 | }); 1250 | } else { 1251 | log.error("[{}] not support shrink topic partitions. old: {}, new: {}", 1252 | topicName, oldPartitionNumber, currentPartitionNumber); 1253 | return FutureUtil.failedFuture(new NotSupportedException("not support shrink topic partitions")); 1254 | } 1255 | }); 1256 | } private final TimerTask partitionsAutoUpdateTimerTask = new TimerTask() { 1257 | @Override 1258 | public void run(Timeout timeout) throws Exception { 1259 | if (timeout.isCancelled() || getState() != State.Ready) { 1260 | return; 1261 | } 1262 | 1263 | if (log.isDebugEnabled()) { 1264 | log.debug("[{}] run partitionsAutoUpdateTimerTask", topic); 1265 | } 1266 | 1267 | // if last auto update not completed yet, do nothing. 1268 | if (partitionsAutoUpdateFuture == null || partitionsAutoUpdateFuture.isDone()) { 1269 | partitionsAutoUpdateFuture = topicsPartitionChangedListener.onTopicsExtended(partitionedTopics.keySet()); 1270 | } 1271 | 1272 | // schedule the next re-check task 1273 | partitionsAutoUpdateTimeout = client.timer() 1274 | .newTimeout(partitionsAutoUpdateTimerTask, conf.getAutoUpdatePartitionsIntervalSeconds(), TimeUnit.SECONDS); 1275 | } 1276 | }; 1277 | 1278 | @VisibleForTesting 1279 | public Timeout getPartitionsAutoUpdateTimeout() { 1280 | return partitionsAutoUpdateTimeout; 1281 | } 1282 | 1283 | @Override 1284 | public CompletableFuture getLastMessageIdAsync() { 1285 | CompletableFuture returnFuture = new CompletableFuture<>(); 1286 | 1287 | Map> messageIdFutures = consumers.entrySet().stream() 1288 | .map(entry -> Pair.of(entry.getKey(), entry.getValue().getLastMessageIdAsync())) 1289 | .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); 1290 | 1291 | CompletableFuture 1292 | .allOf(messageIdFutures.entrySet().stream().map(Map.Entry::getValue).toArray(CompletableFuture[]::new)) 1293 | .whenComplete((ignore, ex) -> { 1294 | ImmutableMap.Builder builder = ImmutableMap.builder(); 1295 | messageIdFutures.forEach((key, future) -> { 1296 | MessageId messageId; 1297 | try { 1298 | messageId = future.get(); 1299 | } catch (Exception e) { 1300 | log.warn("[{}] Exception when topic {} getLastMessageId.", key, e); 1301 | messageId = MessageId.earliest; 1302 | } 1303 | builder.put(key, messageId); 1304 | }); 1305 | returnFuture.complete(new MultiMessageIdImpl(builder.build())); 1306 | }); 1307 | 1308 | return returnFuture; 1309 | } 1310 | 1311 | public void tryAcknowledgeMessage(Message msg) { 1312 | if (msg != null) { 1313 | acknowledgeCumulativeAsync(msg); 1314 | } 1315 | } 1316 | 1317 | // This listener is triggered when topics partitions are updated. 1318 | private class TopicsPartitionChangedListener implements PartitionsChangedListener { 1319 | // Check partitions changes of passed in topics, and subscribe new added partitions. 1320 | @Override 1321 | public CompletableFuture onTopicsExtended(Collection topicsExtended) { 1322 | CompletableFuture future = new CompletableFuture<>(); 1323 | if (topicsExtended.isEmpty()) { 1324 | future.complete(null); 1325 | return future; 1326 | } 1327 | 1328 | if (log.isDebugEnabled()) { 1329 | log.debug("[{}] run onTopicsExtended: {}, size: {}", 1330 | topic, topicsExtended, topicsExtended.size()); 1331 | } 1332 | 1333 | List> futureList = Lists.newArrayListWithExpectedSize(topicsExtended.size()); 1334 | topicsExtended.forEach(topic -> futureList.add(subscribeIncreasedTopicPartitions(topic))); 1335 | FutureUtil.waitForAll(futureList) 1336 | .thenAccept(finalFuture -> future.complete(null)) 1337 | .exceptionally(ex -> { 1338 | log.warn("[{}] Failed to subscribe increased topics partitions: {}", topic, ex.getMessage()); 1339 | future.completeExceptionally(ex); 1340 | return null; 1341 | }); 1342 | 1343 | return future; 1344 | } 1345 | } 1346 | 1347 | 1348 | } 1349 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/WeightedConsumerBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl; 17 | 18 | import org.apache.pulsar.client.api.*; 19 | import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; 20 | import org.apache.pulsar.client.impl.weight.WeightedConsumerConfiguration; 21 | import org.apache.pulsar.client.util.RetryMessageUtil; 22 | import org.apache.pulsar.common.naming.TopicName; 23 | import org.apache.pulsar.common.util.FutureUtil; 24 | import org.apache.pulsar.shade.org.apache.commons.lang3.StringUtils; 25 | 26 | import java.util.List; 27 | import java.util.Map; 28 | import java.util.concurrent.CompletableFuture; 29 | 30 | import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkArgument; 31 | 32 | public class WeightedConsumerBuilder extends ConsumerBuilderImpl { 33 | private WeightedConsumerConfiguration weightConf; 34 | private Integer retryTopicWeight; 35 | 36 | public WeightedConsumerBuilder(PulsarClient client, Schema schema) { 37 | this((PulsarClientImpl) client, new ConsumerConfigurationData(), schema); 38 | } 39 | 40 | WeightedConsumerBuilder(PulsarClientImpl client, ConsumerConfigurationData conf, Schema schema) { 41 | super(client, conf, schema); 42 | weightConf = WeightedConsumerConfiguration.loadFromConf(conf); 43 | } 44 | 45 | public WeightedConsumerConfiguration getWeightConf() { 46 | return weightConf; 47 | } 48 | 49 | @Override 50 | public WeightedConsumerBuilder loadConf(Map config) { 51 | super.loadConf(config); 52 | weightConf = WeightedConsumerConfiguration.loadFromConf(getConf()); 53 | return this; 54 | } 55 | 56 | @Override 57 | public WeightedConsumerBuilder clone() { 58 | return new WeightedConsumerBuilder<>(getClient(), getConf().clone(), getSchema()); 59 | } 60 | 61 | public WeightedConsumerBuilder distributionStrategy(WeightedConsumerConfiguration.DistributionStrategy distributionStrategy) { 62 | weightConf.setDistributionStrategy(distributionStrategy); 63 | return this; 64 | } 65 | 66 | public WeightedConsumerBuilder minBound(int minBound) { 67 | weightConf.setMinBound(minBound); 68 | return this; 69 | } 70 | 71 | public WeightedConsumerBuilder maxBound(int maxBound) { 72 | weightConf.setMaxBound(maxBound); 73 | return this; 74 | } 75 | 76 | public WeightedConsumerBuilder retryTopicWeight(Integer weight) { 77 | if (weight != null) { 78 | checkArgument(weight >= 1 && weight <= weightConf.getMaxWeightAllowed(), 79 | "non-null weights should be in the range [1,maxWeightAllowed(%s)]", weightConf.getMaxWeightAllowed()); 80 | } 81 | this.retryTopicWeight = weight; 82 | return this; 83 | } 84 | 85 | public WeightedConsumerBuilder throttleReceiveQueue(boolean shouldThrotle) { 86 | weightConf.setThrottleReceiveQueue(shouldThrotle); 87 | return this; 88 | } 89 | 90 | public WeightedConsumerBuilder queueResumeThreshold(int resumeThreshold) { 91 | weightConf.setQueueResumeThreshold(resumeThreshold); 92 | return this; 93 | } 94 | 95 | public WeightedConsumerBuilder maxWeightAllowed(int maxWeightAllowed) { 96 | weightConf.setMaxWeightAllowed(maxWeightAllowed); 97 | return this; 98 | } 99 | 100 | //If weight is null, attempt is made to parse it from topic name, otherwise defaults to 1 101 | public WeightedConsumerBuilder topic(String topic, Integer weight) { 102 | weightConf.addTopic(topic, weight); 103 | return this; 104 | } 105 | 106 | public WeightedConsumerBuilder topics(Map topicWeights) { 107 | checkArgument(topicWeights != null && topicWeights.size() > 0, "non-empty topic-weight map required"); 108 | for (Map.Entry entry : topicWeights.entrySet()) { 109 | topic(entry.getKey(), entry.getValue()); 110 | } 111 | return this; 112 | } 113 | private String getSubscriptionName(ConsumerConfigurationData conf){ 114 | String subscriptionName=conf.getSubscriptionName(); 115 | subscriptionName.replace("/", "__"); 116 | return subscriptionName; 117 | } 118 | 119 | 120 | /** 121 | * Copied from {@link ConsumerBuilderImpl#subscribeAsync()} 122 | * Few changes, do a diff to quickly identify those 123 | **/ 124 | @Override 125 | public CompletableFuture> subscribeAsync() { 126 | weightConf.populateBuilder(this); 127 | //Since the members are private in superclass, using getters here to setup relevant local variables 128 | ConsumerConfigurationData conf = getConf(); 129 | PulsarClientImpl client = getClient(); 130 | Schema schema = getSchema(); 131 | List> interceptorList = getInterceptorList(); 132 | 133 | if (conf.getTopicNames().isEmpty() && conf.getTopicsPattern() == null) { 134 | return FutureUtil 135 | .failedFuture(new PulsarClientException.InvalidConfigurationException("Topic name must be set on the consumer builder")); 136 | } 137 | 138 | if (StringUtils.isBlank(conf.getSubscriptionName())) { 139 | return FutureUtil.failedFuture( 140 | new PulsarClientException.InvalidConfigurationException("Subscription name must be set on the consumer builder")); 141 | } 142 | 143 | if (conf.getKeySharedPolicy() != null && conf.getSubscriptionType() != SubscriptionType.Key_Shared) { 144 | return FutureUtil.failedFuture( 145 | new PulsarClientException.InvalidConfigurationException("KeySharedPolicy must set with KeyShared subscription")); 146 | } 147 | if(conf.isRetryEnable() && conf.getTopicNames().size() > 0 ) { 148 | TopicName topicFirst = TopicName.get(conf.getTopicNames().iterator().next()); 149 | String subscriptionName=getSubscriptionName(conf); 150 | String retryLetterTopic = topicFirst.getNamespace() + "/" + subscriptionName + RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; 151 | String deadLetterTopic = topicFirst.getNamespace() + "/" + subscriptionName + RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; 152 | if(conf.getDeadLetterPolicy() == null) { 153 | conf.setDeadLetterPolicy(DeadLetterPolicy.builder() 154 | .maxRedeliverCount(RetryMessageUtil.MAX_RECONSUMETIMES) 155 | .retryLetterTopic(retryLetterTopic) 156 | .deadLetterTopic(deadLetterTopic) 157 | .build()); 158 | } else { 159 | if (StringUtils.isBlank(conf.getDeadLetterPolicy().getRetryLetterTopic())) { 160 | conf.getDeadLetterPolicy().setRetryLetterTopic(retryLetterTopic); 161 | } 162 | if (StringUtils.isBlank(conf.getDeadLetterPolicy().getDeadLetterTopic())) { 163 | conf.getDeadLetterPolicy().setDeadLetterTopic(deadLetterTopic); 164 | } 165 | } 166 | 167 | /** hacky - Populate {@link ConsumerConfigurationData#topicNames} directly since weightConf is not going to be read from at this point **/ 168 | WeightedConsumerConfiguration.populateTopicInBuilder(this, conf.getDeadLetterPolicy().getRetryLetterTopic(), retryTopicWeight); 169 | } 170 | 171 | return interceptorList == null || interceptorList.size() == 0 ? 172 | WeightedMultiTopicsConsumerImpl.subscribeAsync(client, conf, weightConf, schema, null) : 173 | WeightedMultiTopicsConsumerImpl.subscribeAsync(client, conf, weightConf, schema, new ConsumerInterceptors<>(interceptorList)); 174 | } 175 | } 176 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/WeightedMultiTopicsConsumerImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | package org.apache.pulsar.client.impl; 16 | 17 | import org.apache.pulsar.client.api.*; 18 | import org.apache.pulsar.client.api.PulsarClientException.NotSupportedException; 19 | import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; 20 | import org.apache.pulsar.client.impl.transaction.TransactionImpl; 21 | import org.apache.pulsar.client.impl.weight.TopicMessageCountTracker; 22 | import org.apache.pulsar.client.impl.weight.TopicThresholdDistribution; 23 | import org.apache.pulsar.client.impl.weight.TopicThresholdDistributionImpl; 24 | import org.apache.pulsar.client.impl.weight.WeightedConsumerConfiguration; 25 | import org.apache.pulsar.client.util.ConsumerName; 26 | import org.apache.pulsar.client.util.ExecutorProvider; 27 | import org.apache.pulsar.common.api.proto.CommandAck.AckType; 28 | import org.apache.pulsar.common.naming.TopicDomain; 29 | import org.apache.pulsar.common.naming.TopicName; 30 | import org.apache.pulsar.common.partition.PartitionedTopicMetadata; 31 | import org.apache.pulsar.common.util.CompletableFutureCancellationHandler; 32 | import org.apache.pulsar.common.util.FutureUtil; 33 | import org.apache.pulsar.shade.com.google.common.annotations.VisibleForTesting; 34 | import org.apache.pulsar.shade.com.google.common.collect.ImmutableMap; 35 | import org.apache.pulsar.shade.com.google.common.collect.Lists; 36 | import org.apache.pulsar.shade.io.netty.util.Timeout; 37 | import org.apache.pulsar.shade.io.netty.util.TimerTask; 38 | import org.apache.pulsar.shade.org.apache.commons.lang3.tuple.Pair; 39 | import org.slf4j.Logger; 40 | import org.slf4j.LoggerFactory; 41 | 42 | import java.lang.reflect.Field; 43 | import java.util.*; 44 | import java.util.concurrent.*; 45 | import java.util.concurrent.atomic.AtomicBoolean; 46 | import java.util.concurrent.atomic.AtomicInteger; 47 | import java.util.concurrent.atomic.AtomicReference; 48 | import java.util.concurrent.locks.ReadWriteLock; 49 | import java.util.concurrent.locks.ReentrantReadWriteLock; 50 | import java.util.function.Function; 51 | import java.util.stream.Collectors; 52 | import java.util.stream.IntStream; 53 | 54 | import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkArgument; 55 | import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkState; 56 | import static org.apache.pulsar.shade.org.apache.commons.lang3.StringUtils.isBlank; 57 | 58 | public class WeightedMultiTopicsConsumerImpl extends ConsumerBase { 59 | 60 | public static final String DUMMY_TOPIC_NAME_PREFIX = "WeightedMultiConsumerImpl-"; 61 | private static final Logger log = LoggerFactory.getLogger(WeightedMultiTopicsConsumerImpl.class); 62 | // Map , store partition number for each topic 63 | protected final ConcurrentHashMap partitionedTopics; 64 | // Map , when get do ACK, consumer will by find by topic name 65 | private final ConcurrentHashMap> consumers; 66 | // Queue of partition consumers on which we have stopped calling receiveAsync() because the 67 | // topic level thresholds have exceeded 68 | private final ConcurrentLinkedQueue> pausedConsumers; 69 | // Threshold for the shared queue. When the size of the shared queue goes below the threshold, we are going to 70 | // resume receiving from the paused consumer partitions 71 | private final int sharedQueueResumeThreshold; 72 | private final boolean throttleReceiverQueue; 73 | private final TopicThresholdDistribution topicThresholdDistribution; 74 | private final ConcurrentHashMap messageCounters = new ConcurrentHashMap<>(); 75 | private final Object pauseMutex = new Object(); 76 | private final ReadWriteLock lock = new ReentrantReadWriteLock(); 77 | private final ConsumerStatsRecorder stats; 78 | private final UnAckedMessageTracker unAckedMessageTracker; 79 | private final ConsumerConfigurationData internalConfig; 80 | private final long startMessageRollbackDurationInSec; 81 | // sum of topicPartitions, simple topic has 1, partitioned topic equals to partition number. 82 | AtomicInteger allTopicPartitionsNumber; 83 | TopicsPartitionChangedListener topicsPartitionChangedListener; 84 | CompletableFuture partitionsAutoUpdateFuture = null; 85 | private boolean paused = false; 86 | // timeout related to auto check and subscribe partition increasement 87 | private volatile Timeout partitionsAutoUpdateTimeout = null; 88 | private volatile BatchMessageIdImpl startMessageId = null; 89 | 90 | WeightedMultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, 91 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 92 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { 93 | this(client, DUMMY_TOPIC_NAME_PREFIX + ConsumerName.generateRandomName(), conf, executorProvider, 94 | subscribeFuture, schema, interceptors, createTopicIfDoesNotExist); 95 | } 96 | 97 | WeightedMultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, 98 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 99 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, 100 | long startMessageRollbackDurationInSec) { 101 | this(client, DUMMY_TOPIC_NAME_PREFIX + ConsumerName.generateRandomName(), conf, executorProvider, 102 | subscribeFuture, schema, interceptors, createTopicIfDoesNotExist, startMessageId, 103 | startMessageRollbackDurationInSec); 104 | } 105 | 106 | WeightedMultiTopicsConsumerImpl(PulsarClientImpl client, String singleTopic, ConsumerConfigurationData conf, 107 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 108 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { 109 | this(client, singleTopic, conf, executorProvider, subscribeFuture, schema, interceptors, 110 | createTopicIfDoesNotExist, null, 0); 111 | } 112 | 113 | WeightedMultiTopicsConsumerImpl(PulsarClientImpl client, String singleTopic, ConsumerConfigurationData conf, 114 | ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, 115 | ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, 116 | long startMessageRollbackDurationInSec) { 117 | super(client, singleTopic, conf, Math.max(2, conf.getReceiverQueueSize()), executorProvider, subscribeFuture, 118 | schema, interceptors); 119 | 120 | checkArgument(conf.getReceiverQueueSize() > 0, 121 | "Receiver queue size needs to be greater than 0 for Topics Consumer"); 122 | log.info("Receive queue for multi consumer {}", maxReceiverQueueSize); 123 | 124 | this.partitionedTopics = new ConcurrentHashMap<>(); 125 | this.consumers = new ConcurrentHashMap<>(); 126 | this.pausedConsumers = new ConcurrentLinkedQueue<>(); 127 | this.allTopicPartitionsNumber = new AtomicInteger(0); 128 | this.startMessageId = startMessageId != null ? new BatchMessageIdImpl(MessageIdImpl.convertToMessageIdImpl(startMessageId)) : null; 129 | this.startMessageRollbackDurationInSec = startMessageRollbackDurationInSec; 130 | 131 | WeightedConsumerConfiguration weightConf = WeightedConsumerConfiguration.loadFromConf(conf); 132 | log.info("{}", weightConf); 133 | this.topicThresholdDistribution = TopicThresholdDistributionImpl.loadFromConf(weightConf); 134 | this.sharedQueueResumeThreshold = Math.max(0, Math.min(topicThresholdDistribution.getMinValue(), weightConf.getQueueResumeThreshold())); 135 | this.throttleReceiverQueue = weightConf.isThrottleReceiveQueue(); 136 | 137 | if (conf.getAckTimeoutMillis() != 0) { 138 | if (conf.getTickDurationMillis() > 0) { 139 | this.unAckedMessageTracker = new UnAckedTopicMessageTracker(client, this, conf); 140 | } else { 141 | this.unAckedMessageTracker = new UnAckedTopicMessageTracker(client, this, conf); 142 | } 143 | } else { 144 | this.unAckedMessageTracker = UnAckedMessageTracker.UNACKED_MESSAGE_TRACKER_DISABLED; 145 | } 146 | 147 | this.internalConfig = getInternalConsumerConfig(); 148 | this.stats = client.getConfiguration().getStatsIntervalSeconds() > 0 ? new ConsumerStatsRecorderImpl(this) : null; 149 | 150 | // start track and auto subscribe partition increment 151 | if (conf.isAutoUpdatePartitions()) { 152 | topicsPartitionChangedListener = new TopicsPartitionChangedListener(); 153 | partitionsAutoUpdateTimeout = client.timer() 154 | .newTimeout(partitionsAutoUpdateTimerTask, conf.getAutoUpdatePartitionsIntervalSeconds(), TimeUnit.SECONDS); 155 | } 156 | 157 | if (conf.getTopicNames().isEmpty()) { 158 | setState(State.Ready); 159 | subscribeFuture().complete(WeightedMultiTopicsConsumerImpl.this); 160 | return; 161 | } 162 | 163 | checkArgument(conf.getTopicNames().isEmpty() || topicNamesValid(conf.getTopicNames()), "Topics is empty or invalid."); 164 | 165 | List> futures = conf.getTopicNames().stream().map(t -> subscribeAsync(t, createTopicIfDoesNotExist)) 166 | .collect(Collectors.toList()); 167 | FutureUtil.waitForAll(futures) 168 | .thenAccept(finalFuture -> { 169 | if (allTopicPartitionsNumber.get() > maxReceiverQueueSize) { 170 | setMaxReceiverQueueSize(allTopicPartitionsNumber.get()); 171 | } 172 | setState(State.Ready); 173 | // We have successfully created N consumers, so we can start receiving messages now 174 | startReceivingMessages(new ArrayList<>(consumers.values())); 175 | log.info("[{}] [{}] Created topics consumer with {} sub-consumers", 176 | topic, subscription, allTopicPartitionsNumber.get()); 177 | subscribeFuture().complete(WeightedMultiTopicsConsumerImpl.this); 178 | }) 179 | .exceptionally(ex -> { 180 | log.warn("[{}] Failed to subscribe topics: {}, closing consumer", topic, ex.getMessage()); 181 | closeAsync().whenComplete((res, closeEx) -> { 182 | if (closeEx != null) { 183 | log.error("[{}] Failed to unsubscribe after failed consumer creation: {}", topic, closeEx.getMessage()); 184 | } 185 | subscribeFuture.completeExceptionally(ex); 186 | }); 187 | return null; 188 | }); 189 | } 190 | 191 | // Check topics are valid. 192 | // - each topic is valid, 193 | // - topic names are unique. 194 | private static boolean topicNamesValid(Collection topics) { 195 | checkState(topics != null && topics.size() >= 1, 196 | "topics should contain more than 1 topic"); 197 | 198 | Optional result = topics.stream() 199 | .filter(topic -> !TopicName.isValid(topic)) 200 | .findFirst(); 201 | 202 | if (result.isPresent()) { 203 | log.warn("Received invalid topic name: {}", result.get()); 204 | return false; 205 | } 206 | 207 | // check topic names are unique 208 | HashSet set = new HashSet<>(topics); 209 | if (set.size() == topics.size()) { 210 | return true; 211 | } else { 212 | log.warn("Topic names not unique. unique/all : {}/{}", set.size(), topics.size()); 213 | return false; 214 | } 215 | } 216 | 217 | // create consumer for a single topic with already known partitions. 218 | // first create a consumer with no topic, then do subscription for already know partitionedTopic. 219 | public static WeightedMultiTopicsConsumerImpl createPartitionedConsumer(PulsarClientImpl client, 220 | ConsumerConfigurationData conf, 221 | ExecutorProvider executorProvider, 222 | CompletableFuture> subscribeFuture, 223 | int numPartitions, 224 | Schema schema, ConsumerInterceptors interceptors) { 225 | checkArgument(conf.getTopicNames().size() == 1, "Should have only 1 topic for partitioned consumer"); 226 | 227 | // get topic name, then remove it from conf, so constructor will create a consumer with no topic. 228 | ConsumerConfigurationData cloneConf = conf.clone(); 229 | String topicName = cloneConf.getSingleTopic(); 230 | cloneConf.getTopicNames().remove(topicName); 231 | 232 | CompletableFuture future = new CompletableFuture<>(); 233 | WeightedMultiTopicsConsumerImpl consumer = new WeightedMultiTopicsConsumerImpl(client, topicName, cloneConf, executorProvider, 234 | future, schema, interceptors, true /* createTopicIfDoesNotExist */); 235 | 236 | future.thenCompose(c -> ((WeightedMultiTopicsConsumerImpl) c).subscribeAsync(topicName, numPartitions)) 237 | .thenRun(() -> subscribeFuture.complete(consumer)) 238 | .exceptionally(e -> { 239 | log.warn("Failed subscription for createPartitionedConsumer: {} {}, e:{}", 240 | topicName, numPartitions, e); 241 | consumer.cleanupMultiConsumer(); 242 | subscribeFuture.completeExceptionally( 243 | PulsarClientException.wrap(((Throwable) e).getCause(), String.format("Failed to subscribe %s with %d partitions", topicName, numPartitions))); 244 | return null; 245 | }); 246 | return consumer; 247 | } 248 | 249 | public static boolean isIllegalMultiTopicsMessageId(MessageId messageId) { 250 | //only support earliest/latest 251 | return !MessageId.earliest.equals(messageId) && !MessageId.latest.equals(messageId); 252 | } 253 | 254 | public static CompletableFuture> subscribeAsync(PulsarClient client, ConsumerConfigurationData conf, WeightedConsumerConfiguration weightConf, Schema schema, ConsumerInterceptors interceptors) { 255 | if (!(client instanceof PulsarClientImpl)) { 256 | return FutureUtil.failedFuture(new IllegalArgumentException("Expected pulsar client to be of type: " + PulsarClientImpl.class.getName())); 257 | } 258 | PulsarClientImpl clientImpl = (PulsarClientImpl) client; 259 | AtomicReference state = clientImpl.getState(); 260 | 261 | Set> internalClientConsumers = null; 262 | try { 263 | Field consumersField = clientImpl.getClass().getDeclaredField("consumers"); 264 | consumersField.setAccessible(true); 265 | internalClientConsumers = (Set>) consumersField.get(clientImpl); 266 | } catch (NoSuchFieldException | IllegalAccessException ex) { 267 | log.debug("Cannot retrieve internal consumers of the pulsar client, fatal failure"); 268 | return FutureUtil.failedFuture(new PulsarClientException("Cannot retrieve internal consumers of the pulsar client, fatal failure", ex)); 269 | } 270 | 271 | if (state.get() != PulsarClientImpl.State.Open) { 272 | return FutureUtil.failedFuture(new PulsarClientException.AlreadyClosedException("Client already closed")); 273 | } 274 | 275 | if (conf == null) { 276 | return FutureUtil.failedFuture( 277 | new PulsarClientException.InvalidConfigurationException("Consumer configuration undefined")); 278 | } 279 | 280 | for (String topic : conf.getTopicNames()) { 281 | if (!TopicName.isValid(topic)) { 282 | return FutureUtil.failedFuture(new PulsarClientException.InvalidTopicNameException("Invalid topic name: '" + topic + "'")); 283 | } 284 | } 285 | 286 | if (isBlank(conf.getSubscriptionName())) { 287 | return FutureUtil 288 | .failedFuture(new PulsarClientException.InvalidConfigurationException("Empty subscription name")); 289 | } 290 | 291 | if (conf.isReadCompacted() && (!conf.getTopicNames().stream() 292 | .allMatch(topic -> TopicName.get(topic).getDomain() == TopicDomain.persistent) 293 | || (conf.getSubscriptionType() != SubscriptionType.Exclusive 294 | && conf.getSubscriptionType() != SubscriptionType.Failover))) { 295 | return FutureUtil.failedFuture(new PulsarClientException.InvalidConfigurationException( 296 | "Read compacted can only be used with exclusive or failover persistent subscriptions")); 297 | } 298 | 299 | if (conf.getConsumerEventListener() != null && conf.getSubscriptionType() != SubscriptionType.Failover) { 300 | return FutureUtil.failedFuture(new PulsarClientException.InvalidConfigurationException( 301 | "Active consumer listener is only supported for failover subscription")); 302 | } 303 | 304 | conf.setReceiverQueueSize(weightConf.getMaxBound()); 305 | if (conf.getTopicsPattern() != null) { 306 | return FutureUtil.failedFuture(new IllegalArgumentException("Topics pattern not supported in weighted consumer")); 307 | } else { 308 | CompletableFuture> consumerSubscribedFuture = new CompletableFuture<>(); 309 | ConsumerBase consumer = new WeightedMultiTopicsConsumerImpl(clientImpl, conf, 310 | clientImpl.externalExecutorProvider(), consumerSubscribedFuture, schema, interceptors, 311 | true); 312 | internalClientConsumers.add(consumer); 313 | 314 | return consumerSubscribedFuture; 315 | } 316 | } 317 | 318 | private static String partitionTopic(String topic) { 319 | TopicName topicName = TopicName.get(topic); 320 | if (topicName.isPartitioned()) { 321 | topic = topicName.getPartitionedTopicName(); 322 | } 323 | return topic; 324 | } 325 | 326 | private void startReceivingMessages(List> newConsumers) { 327 | if (log.isDebugEnabled()) { 328 | log.debug("[{}] startReceivingMessages for {} new consumers in topics consumer, state: {}", 329 | topic, newConsumers.size(), getState()); 330 | } 331 | 332 | if (getState() == State.Ready) { 333 | newConsumers.forEach(consumer -> { 334 | int permits = getReceiverQueueSize(consumer.getTopic()); 335 | if (log.isDebugEnabled()) { 336 | log.debug("Increasing permits of consumer for topic {} by {}", consumer.getTopic(), permits); 337 | } 338 | consumer.increaseAvailablePermits(consumer.getConnectionHandler().cnx(), permits); 339 | internalPinnedExecutor.execute(() -> receiveMessageFromConsumer(consumer)); 340 | }); 341 | } 342 | } 343 | 344 | private void receiveMessageFromConsumer(ConsumerImpl consumer) { 345 | consumer.receiveAsync().thenAccept(message -> { 346 | if (log.isDebugEnabled()) { 347 | log.debug("[{}] [{}] Receive message from sub consumer:{}", 348 | topic, subscription, consumer.getTopic()); 349 | } 350 | // Process the message, add to the queue and trigger listener or async callback 351 | messageReceived(consumer, message); 352 | 353 | String partitionTopic = partitionTopic(consumer.getTopic()); 354 | int threshold = topicThresholdDistribution.getValue(partitionTopic); 355 | int messagesInQueue = messageCounters.get(partitionTopic).get(); 356 | if (messagesInQueue > threshold) { 357 | // mark this consumer to be resumed later: if No more space left in shared queue 358 | pausedConsumers.add(consumer); 359 | if (log.isDebugEnabled()) { 360 | log.debug("Consumer {} paused with state {}/{} and total={}", topic, messagesInQueue, threshold, incomingMessages.size()); 361 | } 362 | // Since we din't get a mutex, the condition on the incoming queue might have changed after 363 | // we have paused the current consumer. We need to re-check in order to avoid this consumer 364 | // from getting stalled. 365 | resumeReceivingFromPausedConsumersIfNeeded(); 366 | } else { 367 | // Schedule next receiveAsync() if the incoming queue is not full. Use a different thread to avoid 368 | // recursion and stack overflow 369 | internalPinnedExecutor.execute(() -> receiveMessageFromConsumer(consumer)); 370 | } 371 | }); 372 | } 373 | 374 | private void messageReceived(ConsumerImpl consumer, Message message) { 375 | checkArgument(message instanceof MessageImpl); 376 | TopicMessageImpl topicMessage = new TopicMessageImpl<>(consumer.getTopic(), 377 | consumer.getTopicNameWithoutPartition(), message, consumer); 378 | 379 | if (log.isDebugEnabled()) { 380 | log.debug("[{}][{}] Received message from topics-consumer {}", 381 | topic, subscription, message.getMessageId()); 382 | } 383 | 384 | // if asyncReceive is waiting : return message to callback without adding to incomingMessages queue 385 | CompletableFuture> receivedFuture = nextPendingReceive(); 386 | if (receivedFuture != null) { 387 | unAckedMessageTracker.add(topicMessage.getMessageId()); 388 | completePendingReceive(receivedFuture, topicMessage); 389 | } else if (enqueueMessageAndCheckBatchReceive(topicMessage) && hasPendingBatchReceive()) { 390 | notifyPendingBatchReceivedCallBack(); 391 | } 392 | 393 | if (listener != null) { 394 | triggerListener(); 395 | } 396 | } 397 | 398 | @Override 399 | protected synchronized void messageProcessed(Message msg) { 400 | unAckedMessageTracker.add(msg.getMessageId()); 401 | decreaseIncomingMessageSize(msg); 402 | } 403 | 404 | private void resumeReceivingFromPausedConsumersIfNeeded() { 405 | if (incomingMessages.size() <= sharedQueueResumeThreshold && !pausedConsumers.isEmpty()) { 406 | while (true) { 407 | ConsumerImpl consumer = pausedConsumers.poll(); 408 | if (consumer == null) { 409 | break; 410 | } 411 | 412 | internalPinnedExecutor.execute(() -> { 413 | receiveMessageFromConsumer(consumer); 414 | }); 415 | } 416 | } 417 | } 418 | 419 | @Override 420 | protected boolean enqueueMessageAndCheckBatchReceive(Message message) { 421 | Boolean val=super.enqueueMessageAndCheckBatchReceive(message); 422 | messageCounters.get(partitionTopic(message.getTopicName())).incrementAndGet(); 423 | return val; 424 | } 425 | 426 | @Override 427 | protected void decreaseIncomingMessageSize(Message message) { 428 | super.decreaseIncomingMessageSize(message); 429 | messageCounters.get(partitionTopic(message.getTopicName())).decrementAndGet(); 430 | } 431 | 432 | @Override 433 | protected void resetIncomingMessageSize() { 434 | super.resetIncomingMessageSize(); 435 | 436 | TopicMessageCountTracker topicMessageCountTracker = new TopicMessageCountTracker(); 437 | incomingMessages.drainTo(topicMessageCountTracker); 438 | topicMessageCountTracker.getCounters().forEach((topic, count) -> 439 | messageCounters.get(partitionTopic(topic)).addAndGet(-1 * count)); 440 | } 441 | 442 | protected void addMessageCounterForTopic(String topic) { 443 | messageCounters.putIfAbsent(partitionTopic(topic), new AtomicInteger()); 444 | } 445 | 446 | @Override 447 | protected Message internalReceive() throws PulsarClientException { 448 | Message message; 449 | try { 450 | message = incomingMessages.take(); 451 | decreaseIncomingMessageSize(message); 452 | checkState(message instanceof TopicMessageImpl); 453 | unAckedMessageTracker.add(message.getMessageId()); 454 | resumeReceivingFromPausedConsumersIfNeeded(); 455 | return message; 456 | } catch (Exception e) { 457 | throw PulsarClientException.unwrap(e); 458 | } 459 | } 460 | 461 | @Override 462 | protected Message internalReceive(int timeout, TimeUnit unit) throws PulsarClientException { 463 | Message message; 464 | try { 465 | message = incomingMessages.poll(timeout, unit); 466 | if (message != null) { 467 | decreaseIncomingMessageSize(message); 468 | checkArgument(message instanceof TopicMessageImpl); 469 | unAckedMessageTracker.add(message.getMessageId()); 470 | } 471 | resumeReceivingFromPausedConsumersIfNeeded(); 472 | return message; 473 | } catch (Exception e) { 474 | throw PulsarClientException.unwrap(e); 475 | } 476 | } 477 | 478 | @Override 479 | protected Messages internalBatchReceive() throws PulsarClientException { 480 | try { 481 | return internalBatchReceiveAsync().get(); 482 | } catch (InterruptedException | ExecutionException e) { 483 | State state = getState(); 484 | if (state != State.Closing && state != State.Closed) { 485 | stats.incrementNumBatchReceiveFailed(); 486 | throw PulsarClientException.unwrap(e); 487 | } else { 488 | return null; 489 | } 490 | } 491 | } 492 | 493 | @Override 494 | protected CompletableFuture> internalBatchReceiveAsync() { 495 | CompletableFutureCancellationHandler cancellationHandler = new CompletableFutureCancellationHandler(); 496 | CompletableFuture> result = cancellationHandler.createFuture(); 497 | try { 498 | lock.writeLock().lock(); 499 | if (hasEnoughMessagesForBatchReceive()) { 500 | MessagesImpl messages = getNewMessagesImpl(); 501 | Message msgPeeked = incomingMessages.peek(); 502 | while (msgPeeked != null && messages.canAdd(msgPeeked)) { 503 | Message msg = incomingMessages.poll(); 504 | if (msg != null) { 505 | decreaseIncomingMessageSize(msg); 506 | Message interceptMsg = beforeConsume(msg); 507 | messages.add(interceptMsg); 508 | } 509 | msgPeeked = incomingMessages.peek(); 510 | } 511 | result.complete(messages); 512 | } else { 513 | OpBatchReceive opBatchReceive = OpBatchReceive.of(result); 514 | pendingBatchReceives.add(opBatchReceive); 515 | cancellationHandler.setCancelAction(() -> pendingBatchReceives.remove(opBatchReceive)); 516 | } 517 | resumeReceivingFromPausedConsumersIfNeeded(); 518 | } finally { 519 | lock.writeLock().unlock(); 520 | } 521 | return result; 522 | } 523 | 524 | @Override 525 | protected CompletableFuture> internalReceiveAsync() { 526 | CompletableFutureCancellationHandler cancellationHandler = new CompletableFutureCancellationHandler(); 527 | CompletableFuture> result = cancellationHandler.createFuture(); 528 | Message message = incomingMessages.poll(); 529 | if (message == null) { 530 | pendingReceives.add(result); 531 | cancellationHandler.setCancelAction(() -> pendingReceives.remove(result)); 532 | } else { 533 | decreaseIncomingMessageSize(message); 534 | checkState(message instanceof TopicMessageImpl); 535 | unAckedMessageTracker.add(message.getMessageId()); 536 | resumeReceivingFromPausedConsumersIfNeeded(); 537 | result.complete(message); 538 | } 539 | return result; 540 | } 541 | 542 | @Override 543 | protected CompletableFuture doAcknowledge(MessageId messageId, AckType ackType, 544 | Map properties, 545 | TransactionImpl txnImpl) { 546 | checkArgument(messageId instanceof TopicMessageIdImpl); 547 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 548 | 549 | if (getState() != State.Ready) { 550 | return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); 551 | } 552 | 553 | if (ackType == AckType.Cumulative) { 554 | Consumer individualConsumer = consumers.get(topicMessageId.getTopicPartitionName()); 555 | if (individualConsumer != null) { 556 | MessageId innerId = topicMessageId.getInnerMessageId(); 557 | return individualConsumer.acknowledgeCumulativeAsync(innerId); 558 | } else { 559 | return FutureUtil.failedFuture(new PulsarClientException.NotConnectedException()); 560 | } 561 | } else { 562 | ConsumerImpl consumer = consumers.get(topicMessageId.getTopicPartitionName()); 563 | 564 | MessageId innerId = topicMessageId.getInnerMessageId(); 565 | return consumer.doAcknowledgeWithTxn(innerId, ackType, properties, txnImpl) 566 | .thenRun(() -> 567 | unAckedMessageTracker.remove(topicMessageId)); 568 | } 569 | } 570 | 571 | @Override 572 | protected CompletableFuture doAcknowledge(List messageIdList, AckType ackType, Map properties, TransactionImpl txn) { 573 | List> resultFutures = new ArrayList<>(); 574 | if (ackType == AckType.Cumulative) { 575 | messageIdList.forEach(messageId -> resultFutures.add(doAcknowledge(messageId, ackType, properties, txn))); 576 | return CompletableFuture.allOf(resultFutures.toArray(new CompletableFuture[0])); 577 | } else { 578 | if (getState() != State.Ready) { 579 | return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); 580 | } 581 | Map> topicToMessageIdMap = new HashMap<>(); 582 | for (MessageId messageId : messageIdList) { 583 | if (!(messageId instanceof TopicMessageIdImpl)) { 584 | return FutureUtil.failedFuture(new IllegalArgumentException("messageId is not instance of TopicMessageIdImpl")); 585 | } 586 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 587 | topicToMessageIdMap.putIfAbsent(topicMessageId.getTopicPartitionName(), new ArrayList<>()); 588 | topicToMessageIdMap.get(topicMessageId.getTopicPartitionName()).add(topicMessageId.getInnerMessageId()); 589 | } 590 | topicToMessageIdMap.forEach((topicPartitionName, messageIds) -> { 591 | ConsumerImpl consumer = consumers.get(topicPartitionName); 592 | resultFutures.add(consumer.doAcknowledgeWithTxn(messageIds, ackType, properties, txn) 593 | .thenAccept((res) -> messageIdList.forEach(unAckedMessageTracker::remove))); 594 | }); 595 | return CompletableFuture.allOf(resultFutures.toArray(new CompletableFuture[0])); 596 | } 597 | } 598 | 599 | @Override 600 | protected CompletableFuture doReconsumeLater(Message message, AckType ackType, 601 | Map properties, 602 | long delayTime, 603 | TimeUnit unit) { 604 | MessageId messageId = message.getMessageId(); 605 | checkArgument(messageId instanceof TopicMessageIdImpl); 606 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 607 | if (getState() != State.Ready) { 608 | return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); 609 | } 610 | 611 | if (ackType == AckType.Cumulative) { 612 | Consumer individualConsumer = consumers.get(topicMessageId.getTopicPartitionName()); 613 | if (individualConsumer != null) { 614 | MessageId innerId = topicMessageId.getInnerMessageId(); 615 | return individualConsumer.reconsumeLaterCumulativeAsync(message, delayTime, unit); 616 | } else { 617 | return FutureUtil.failedFuture(new PulsarClientException.NotConnectedException()); 618 | } 619 | } else { 620 | ConsumerImpl consumer = consumers.get(topicMessageId.getTopicPartitionName()); 621 | MessageId innerId = topicMessageId.getInnerMessageId(); 622 | return consumer.doReconsumeLater(message, ackType, properties, delayTime, unit) 623 | .thenRun(() -> unAckedMessageTracker.remove(topicMessageId)); 624 | } 625 | } 626 | 627 | @Override 628 | public void negativeAcknowledge(MessageId messageId) { 629 | checkArgument(messageId instanceof TopicMessageIdImpl); 630 | TopicMessageIdImpl topicMessageId = (TopicMessageIdImpl) messageId; 631 | 632 | ConsumerImpl consumer = consumers.get(topicMessageId.getTopicPartitionName()); 633 | consumer.negativeAcknowledge(topicMessageId.getInnerMessageId()); 634 | } 635 | 636 | @Override 637 | public CompletableFuture unsubscribeAsync() { 638 | if (getState() == State.Closing || getState() == State.Closed) { 639 | return FutureUtil.failedFuture( 640 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 641 | } 642 | setState(State.Closing); 643 | 644 | CompletableFuture unsubscribeFuture = new CompletableFuture<>(); 645 | List> futureList = consumers.values().stream() 646 | .map(c -> c.unsubscribeAsync()).collect(Collectors.toList()); 647 | 648 | FutureUtil.waitForAll(futureList) 649 | .thenCompose((r) -> { 650 | setState(State.Closed); 651 | cleanupMultiConsumer(); 652 | log.info("[{}] [{}] [{}] Unsubscribed Topics Consumer", 653 | topic, subscription, consumerName); 654 | // fail all pending-receive futures to notify application 655 | return failPendingReceive(); 656 | }) 657 | .whenComplete((r, ex) -> { 658 | if (ex == null) { 659 | unsubscribeFuture.complete(null); 660 | } else { 661 | setState(State.Failed); 662 | unsubscribeFuture.completeExceptionally(ex); 663 | log.error("[{}] [{}] [{}] Could not unsubscribe Topics Consumer", 664 | topic, subscription, consumerName, ex.getCause()); 665 | } 666 | }); 667 | 668 | return unsubscribeFuture; 669 | } 670 | 671 | @Override 672 | public CompletableFuture closeAsync() { 673 | if (getState() == State.Closing || getState() == State.Closed) { 674 | unAckedMessageTracker.close(); 675 | return CompletableFuture.completedFuture(null); 676 | } 677 | setState(State.Closing); 678 | 679 | if (partitionsAutoUpdateTimeout != null) { 680 | partitionsAutoUpdateTimeout.cancel(); 681 | partitionsAutoUpdateTimeout = null; 682 | } 683 | 684 | CompletableFuture closeFuture = new CompletableFuture<>(); 685 | List> futureList = consumers.values().stream() 686 | .map(c -> c.closeAsync()).collect(Collectors.toList()); 687 | 688 | FutureUtil.waitForAll(futureList) 689 | .thenCompose((r) -> { 690 | setState(State.Closed); 691 | cleanupMultiConsumer(); 692 | log.info("[{}] [{}] Closed Topics Consumer", topic, subscription); 693 | // fail all pending-receive futures to notify application 694 | return failPendingReceive(); 695 | }) 696 | .whenComplete((r, ex) -> { 697 | if (ex == null) { 698 | closeFuture.complete(null); 699 | } else { 700 | setState(State.Failed); 701 | closeFuture.completeExceptionally(ex); 702 | log.error("[{}] [{}] Could not close Topics Consumer", topic, subscription, 703 | ex.getCause()); 704 | } 705 | }); 706 | 707 | return closeFuture; 708 | } 709 | 710 | private void cleanupMultiConsumer() { 711 | unAckedMessageTracker.close(); 712 | client.cleanupConsumer(this); 713 | } 714 | 715 | @Override 716 | public boolean isConnected() { 717 | return consumers.values().stream().allMatch(consumer -> consumer.isConnected()); 718 | } 719 | 720 | @Override 721 | String getHandlerName() { 722 | return subscription; 723 | } 724 | 725 | private ConsumerConfigurationData getInternalConsumerConfig() { 726 | ConsumerConfigurationData internalConsumerConfig = conf.clone(); 727 | internalConsumerConfig.setSubscriptionName(subscription); 728 | internalConsumerConfig.setConsumerName(consumerName); 729 | internalConsumerConfig.setMessageListener(null); 730 | return internalConsumerConfig; 731 | } 732 | 733 | @Override 734 | public void redeliverUnacknowledgedMessages() { 735 | lock.writeLock().lock(); 736 | try { 737 | consumers.values().stream().forEach(consumer -> { 738 | consumer.redeliverUnacknowledgedMessages(); 739 | consumer.unAckedChunkedMessageIdSequenceMap.clear(); 740 | }); 741 | resetIncomingMessageSize(); 742 | unAckedMessageTracker.clear(); 743 | } finally { 744 | lock.writeLock().unlock(); 745 | } 746 | resumeReceivingFromPausedConsumersIfNeeded(); 747 | } 748 | 749 | @Override 750 | public void redeliverUnacknowledgedMessages(Set messageIds) { 751 | if (messageIds.isEmpty()) { 752 | return; 753 | } 754 | 755 | checkArgument(messageIds.stream().findFirst().get() instanceof TopicMessageIdImpl); 756 | 757 | if (conf.getSubscriptionType() != SubscriptionType.Shared) { 758 | // We cannot redeliver single messages if subscription type is not Shared 759 | redeliverUnacknowledgedMessages(); 760 | return; 761 | } 762 | removeExpiredMessagesFromQueue(messageIds); 763 | messageIds.stream().map(messageId -> (TopicMessageIdImpl) messageId) 764 | .collect(Collectors.groupingBy(TopicMessageIdImpl::getTopicPartitionName, Collectors.toSet())) 765 | .forEach((topicName, messageIds1) -> 766 | consumers.get(topicName) 767 | .redeliverUnacknowledgedMessages(messageIds1.stream() 768 | .map(mid -> mid.getInnerMessageId()).collect(Collectors.toSet()))); 769 | resumeReceivingFromPausedConsumersIfNeeded(); 770 | } 771 | 772 | @Override 773 | protected void completeOpBatchReceive(OpBatchReceive op) { 774 | notifyPendingBatchReceivedCallBack(op); 775 | resumeReceivingFromPausedConsumersIfNeeded(); 776 | } 777 | 778 | @Override 779 | public void seek(MessageId messageId) throws PulsarClientException { 780 | try { 781 | seekAsync(messageId).get(); 782 | } catch (Exception e) { 783 | throw PulsarClientException.unwrap(e); 784 | } 785 | } 786 | 787 | @Override 788 | public void seek(long timestamp) throws PulsarClientException { 789 | try { 790 | seekAsync(timestamp).get(); 791 | } catch (Exception e) { 792 | throw PulsarClientException.unwrap(e); 793 | } 794 | } 795 | 796 | @Override 797 | public void seek(Function function) throws PulsarClientException { 798 | try { 799 | this.seekAsync(function).get(); 800 | } catch (Exception var3) { 801 | throw PulsarClientException.unwrap(var3); 802 | } 803 | } 804 | 805 | @Override 806 | public CompletableFuture seekAsync(Function function) { 807 | List> futures = new ArrayList(this.consumers.size()); 808 | this.consumers.values().forEach((consumer) -> { 809 | futures.add(consumer.seekAsync(function)); 810 | }); 811 | this.unAckedMessageTracker.clear(); 812 | this.resetIncomingMessageSize(); 813 | return FutureUtil.waitForAll(futures); 814 | } 815 | 816 | 817 | @Override 818 | public CompletableFuture seekAsync(MessageId messageId) { 819 | CompletableFuture seekFuture = new CompletableFuture<>(); 820 | MessageIdImpl targetMessageId = MessageIdImpl.convertToMessageIdImpl(messageId); 821 | if (targetMessageId == null || isIllegalMultiTopicsMessageId(messageId)) { 822 | seekFuture.completeExceptionally( 823 | new PulsarClientException("Illegal messageId, messageId can only be earliest/latest")); 824 | return seekFuture; 825 | } 826 | List> futures = new ArrayList<>(consumers.size()); 827 | consumers.values().forEach(consumerImpl -> futures.add(consumerImpl.seekAsync(targetMessageId))); 828 | 829 | unAckedMessageTracker.clear(); 830 | resetIncomingMessageSize(); 831 | 832 | FutureUtil.waitForAll(futures).whenComplete((result, exception) -> { 833 | if (exception != null) { 834 | seekFuture.completeExceptionally(exception); 835 | } else { 836 | seekFuture.complete(result); 837 | } 838 | }); 839 | return seekFuture; 840 | } 841 | 842 | @Override 843 | public CompletableFuture seekAsync(long timestamp) { 844 | List> futures = new ArrayList<>(consumers.size()); 845 | consumers.values().forEach(consumer -> futures.add(consumer.seekAsync(timestamp))); 846 | return FutureUtil.waitForAll(futures); 847 | } 848 | 849 | @Override 850 | public int getAvailablePermits() { 851 | return consumers.values().stream().mapToInt(ConsumerImpl::getAvailablePermits).sum(); 852 | } 853 | 854 | @Override 855 | public boolean hasReachedEndOfTopic() { 856 | return consumers.values().stream().allMatch(Consumer::hasReachedEndOfTopic); 857 | } 858 | 859 | public boolean hasMessageAvailable() throws PulsarClientException { 860 | try { 861 | return hasMessageAvailableAsync().get(); 862 | } catch (Exception e) { 863 | throw PulsarClientException.unwrap(e); 864 | } 865 | } 866 | 867 | public CompletableFuture hasMessageAvailableAsync() { 868 | List> futureList = new ArrayList<>(); 869 | final AtomicBoolean hasMessageAvailable = new AtomicBoolean(false); 870 | for (ConsumerImpl consumer : consumers.values()) { 871 | futureList.add(consumer.hasMessageAvailableAsync().thenAccept(isAvailable -> { 872 | if (isAvailable) { 873 | hasMessageAvailable.compareAndSet(false, true); 874 | } 875 | })); 876 | } 877 | CompletableFuture completableFuture = new CompletableFuture<>(); 878 | FutureUtil.waitForAll(futureList).whenComplete((result, exception) -> { 879 | if (exception != null) { 880 | completableFuture.completeExceptionally(exception); 881 | } else { 882 | completableFuture.complete(hasMessageAvailable.get()); 883 | } 884 | }); 885 | return completableFuture; 886 | } 887 | 888 | @Override 889 | public int numMessagesInQueue() { 890 | return incomingMessages.size() + consumers.values().stream().mapToInt(ConsumerImpl::numMessagesInQueue).sum(); 891 | } 892 | 893 | @Override 894 | public synchronized ConsumerStats getStats() { 895 | if (stats == null) { 896 | return null; 897 | } 898 | stats.reset(); 899 | 900 | consumers.values().stream().forEach(consumer -> stats.updateCumulativeStats(consumer.getStats())); 901 | return stats; 902 | } 903 | 904 | public UnAckedMessageTracker getUnAckedMessageTracker() { 905 | return unAckedMessageTracker; 906 | } 907 | 908 | private void removeExpiredMessagesFromQueue(Set messageIds) { 909 | Message peek = incomingMessages.peek(); 910 | if (peek != null) { 911 | if (!messageIds.contains(peek.getMessageId())) { 912 | // first message is not expired, then no message is expired in queue. 913 | return; 914 | } 915 | 916 | // try not to remove elements that are added while we remove 917 | Message message = incomingMessages.poll(); 918 | checkState(message instanceof TopicMessageImpl); 919 | while (message != null) { 920 | decreaseIncomingMessageSize(message); 921 | MessageId messageId = message.getMessageId(); 922 | if (!messageIds.contains(messageId)) { 923 | messageIds.add(messageId); 924 | break; 925 | } 926 | message = incomingMessages.poll(); 927 | } 928 | } 929 | } 930 | 931 | private TopicName getTopicName(String topic) { 932 | try { 933 | return TopicName.get(topic); 934 | } catch (Exception ignored) { 935 | return null; 936 | } 937 | } 938 | 939 | private String getFullTopicName(String topic) { 940 | TopicName topicName = getTopicName(topic); 941 | return (topicName != null) ? topicName.toString() : null; 942 | } 943 | 944 | private void removeTopic(String topic) { 945 | String fullTopicName = getFullTopicName(topic); 946 | if (fullTopicName != null) { 947 | partitionedTopics.remove(topic); 948 | } 949 | } 950 | 951 | // subscribe one more given topic 952 | public CompletableFuture subscribeAsync(String topicName, boolean createTopicIfDoesNotExist) { 953 | TopicName topicNameInstance = getTopicName(topicName); 954 | if (topicNameInstance == null) { 955 | return FutureUtil.failedFuture( 956 | new PulsarClientException.AlreadyClosedException("Topic name not valid")); 957 | } 958 | String fullTopicName = topicNameInstance.toString(); 959 | if (consumers.containsKey(fullTopicName) || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { 960 | return FutureUtil.failedFuture( 961 | new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); 962 | } 963 | 964 | if (getState() == State.Closing || getState() == State.Closed) { 965 | return FutureUtil.failedFuture( 966 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 967 | } 968 | 969 | CompletableFuture subscribeResult = new CompletableFuture<>(); 970 | 971 | client.getPartitionedTopicMetadata(topicName) 972 | .thenAccept(metadata -> subscribeTopicPartitions(subscribeResult, fullTopicName, metadata.partitions, 973 | createTopicIfDoesNotExist)) 974 | .exceptionally(ex1 -> { 975 | log.warn("[{}] Failed to get partitioned topic metadata: {}", fullTopicName, ex1.getMessage()); 976 | subscribeResult.completeExceptionally(ex1); 977 | return null; 978 | }); 979 | 980 | return subscribeResult; 981 | } 982 | 983 | // subscribe one more given topic, but already know the numberPartitions 984 | @VisibleForTesting 985 | CompletableFuture subscribeAsync(String topicName, int numberPartitions) { 986 | TopicName topicNameInstance = getTopicName(topicName); 987 | if (topicNameInstance == null) { 988 | return FutureUtil.failedFuture( 989 | new PulsarClientException.AlreadyClosedException("Topic name not valid")); 990 | } 991 | String fullTopicName = topicNameInstance.toString(); 992 | if (consumers.containsKey(fullTopicName) || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { 993 | return FutureUtil.failedFuture( 994 | new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); 995 | } 996 | 997 | if (getState() == State.Closing || getState() == State.Closed) { 998 | return FutureUtil.failedFuture( 999 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 1000 | } 1001 | 1002 | CompletableFuture subscribeResult = new CompletableFuture<>(); 1003 | subscribeTopicPartitions(subscribeResult, fullTopicName, numberPartitions, true /* createTopicIfDoesNotExist */); 1004 | 1005 | return subscribeResult; 1006 | } 1007 | 1008 | private void subscribeTopicPartitions(CompletableFuture subscribeResult, String topicName, int numPartitions, 1009 | boolean createIfDoesNotExist) { 1010 | client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((schema, cause) -> { 1011 | if (null == cause) { 1012 | doSubscribeTopicPartitions(schema, subscribeResult, topicName, numPartitions, createIfDoesNotExist); 1013 | } else { 1014 | subscribeResult.completeExceptionally(cause); 1015 | } 1016 | }); 1017 | } 1018 | 1019 | private void doSubscribeTopicPartitions(Schema schema, 1020 | CompletableFuture subscribeResult, String topicName, int numPartitions, 1021 | boolean createIfDoesNotExist) { 1022 | if (log.isDebugEnabled()) { 1023 | log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); 1024 | } 1025 | 1026 | List>> futureList; 1027 | if (numPartitions != PartitionedTopicMetadata.NON_PARTITIONED) { 1028 | // Below condition is true if subscribeAsync() has been invoked second time with same 1029 | // topicName before the first invocation had reached this point. 1030 | boolean isTopicBeingSubscribedForInOtherThread = this.partitionedTopics.putIfAbsent(topicName, numPartitions) != null; 1031 | if (isTopicBeingSubscribedForInOtherThread) { 1032 | String errorMessage = String.format("[%s] Failed to subscribe for topic [%s] in topics consumer. " 1033 | + "Topic is already being subscribed for in other thread.", topic, topicName); 1034 | log.warn(errorMessage); 1035 | subscribeResult.completeExceptionally(new PulsarClientException(errorMessage)); 1036 | return; 1037 | } 1038 | allTopicPartitionsNumber.addAndGet(numPartitions); 1039 | 1040 | ConsumerConfigurationData configurationData = getInternalConsumerConfig(); 1041 | try { 1042 | int receiverQueueSize = getReceiverQueueSize(topicName); 1043 | configurationData.setReceiverQueueSize(receiverQueueSize); 1044 | } catch (Exception ex) { 1045 | subscribeResult.completeExceptionally(new PulsarClientException(ex)); 1046 | return; 1047 | } 1048 | 1049 | futureList = IntStream 1050 | .range(0, numPartitions) 1051 | .mapToObj( 1052 | partitionIndex -> { 1053 | String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); 1054 | CompletableFuture> subFuture = new CompletableFuture<>(); 1055 | ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl(client, partitionName, 1056 | configurationData, client.externalExecutorProvider(), 1057 | partitionIndex, true, subFuture, 1058 | startMessageId, schema, interceptors, 1059 | createIfDoesNotExist, startMessageRollbackDurationInSec); 1060 | addMessageCounterForTopic(newConsumer.getTopic()); 1061 | consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); 1062 | return subFuture; 1063 | }) 1064 | .collect(Collectors.toList()); 1065 | } else { 1066 | allTopicPartitionsNumber.incrementAndGet(); 1067 | 1068 | CompletableFuture> subFuture = new CompletableFuture<>(); 1069 | ConsumerConfigurationData configurationData = internalConfig.clone(); 1070 | try { 1071 | int receiverQueueSize = getReceiverQueueSize(topicName); 1072 | configurationData.setReceiverQueueSize(receiverQueueSize); 1073 | } catch (Exception ex) { 1074 | subscribeResult.completeExceptionally(new PulsarClientException(ex)); 1075 | return; 1076 | } 1077 | consumers.compute(topicName, (key, existingValue) -> { 1078 | if (existingValue != null) { 1079 | String errorMessage = String.format("[%s] Failed to subscribe for topic [%s] in topics consumer. " 1080 | + "Topic is already being subscribed for in other thread.", topic, topicName); 1081 | log.warn(errorMessage); 1082 | subscribeResult.completeExceptionally(new PulsarClientException(errorMessage)); 1083 | return existingValue; 1084 | } else { 1085 | ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl(client, topicName, configurationData, 1086 | client.externalExecutorProvider(), -1, 1087 | true, subFuture, null, schema, interceptors, 1088 | createIfDoesNotExist); 1089 | addMessageCounterForTopic(newConsumer.getTopic()); 1090 | synchronized (pauseMutex) { 1091 | if (paused) { 1092 | newConsumer.pause(); 1093 | } 1094 | } 1095 | return newConsumer; 1096 | } 1097 | }); 1098 | 1099 | futureList = Collections.singletonList(subFuture); 1100 | } 1101 | 1102 | FutureUtil.waitForAll(futureList) 1103 | .thenAccept(finalFuture -> { 1104 | if (allTopicPartitionsNumber.get() > maxReceiverQueueSize) { 1105 | setMaxReceiverQueueSize(allTopicPartitionsNumber.get()); 1106 | } 1107 | 1108 | // We have successfully created new consumers, so we can start receiving messages for them 1109 | startReceivingMessages(consumers.values().stream() 1110 | .filter(consumer1 -> { 1111 | String consumerTopicName = consumer1.getTopic(); 1112 | return TopicName.get(consumerTopicName).getPartitionedTopicName().equals( 1113 | TopicName.get(topicName).getPartitionedTopicName()); 1114 | }) 1115 | .collect(Collectors.toList())); 1116 | 1117 | subscribeResult.complete(null); 1118 | log.info("[{}] [{}] Success subscribe new topic {} in topics consumer, partitions: {}, allTopicPartitionsNumber: {}", 1119 | topic, subscription, topicName, numPartitions, allTopicPartitionsNumber.get()); 1120 | return; 1121 | }) 1122 | .exceptionally(ex -> { 1123 | handleSubscribeOneTopicError(topicName, ex, subscribeResult); 1124 | return null; 1125 | }); 1126 | } 1127 | 1128 | private int getReceiverQueueSize(String topicName) { 1129 | int receiverQueueSize = throttleReceiverQueue 1130 | ? topicThresholdDistribution.getValue(topicName) 1131 | : topicThresholdDistribution.getMaxValue(); 1132 | return receiverQueueSize; 1133 | } 1134 | 1135 | // handling failure during subscribe new topic, unsubscribe success created partitions 1136 | private void handleSubscribeOneTopicError(String topicName, Throwable error, CompletableFuture subscribeFuture) { 1137 | log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, error.getMessage()); 1138 | client.externalExecutorProvider().getExecutor().submit(() -> { 1139 | AtomicInteger toCloseNum = new AtomicInteger(0); 1140 | consumers.values().stream().filter(consumer1 -> { 1141 | String consumerTopicName = consumer1.getTopic(); 1142 | if (TopicName.get(consumerTopicName).getPartitionedTopicName().equals(TopicName.get(topicName).getPartitionedTopicName())) { 1143 | toCloseNum.incrementAndGet(); 1144 | return true; 1145 | } else { 1146 | return false; 1147 | } 1148 | }).collect(Collectors.toList()).forEach(consumer2 -> { 1149 | consumer2.closeAsync().whenComplete((r, ex) -> { 1150 | consumer2.subscribeFuture().completeExceptionally(error); 1151 | allTopicPartitionsNumber.decrementAndGet(); 1152 | consumers.remove(consumer2.getTopic()); 1153 | if (toCloseNum.decrementAndGet() == 0) { 1154 | log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer, subscribe error: {}", 1155 | topic, topicName, error.getMessage()); 1156 | removeTopic(topicName); 1157 | subscribeFuture.completeExceptionally(error); 1158 | } 1159 | return; 1160 | }); 1161 | }); 1162 | }); 1163 | } 1164 | 1165 | // un-subscribe a given topic 1166 | public CompletableFuture unsubscribeAsync(String topicName) { 1167 | checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); 1168 | 1169 | if (getState() == State.Closing || getState() == State.Closed) { 1170 | return FutureUtil.failedFuture( 1171 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 1172 | } 1173 | 1174 | if (partitionsAutoUpdateTimeout != null) { 1175 | partitionsAutoUpdateTimeout.cancel(); 1176 | partitionsAutoUpdateTimeout = null; 1177 | } 1178 | 1179 | CompletableFuture unsubscribeFuture = new CompletableFuture<>(); 1180 | String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); 1181 | 1182 | List> consumersToUnsub = consumers.values().stream() 1183 | .filter(consumer -> { 1184 | String consumerTopicName = consumer.getTopic(); 1185 | return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); 1186 | }).collect(Collectors.toList()); 1187 | 1188 | List> futureList = consumersToUnsub.stream() 1189 | .map(ConsumerImpl::unsubscribeAsync).collect(Collectors.toList()); 1190 | 1191 | FutureUtil.waitForAll(futureList) 1192 | .whenComplete((r, ex) -> { 1193 | if (ex == null) { 1194 | consumersToUnsub.forEach(consumer1 -> { 1195 | consumers.remove(consumer1.getTopic()); 1196 | pausedConsumers.remove(consumer1); 1197 | allTopicPartitionsNumber.decrementAndGet(); 1198 | }); 1199 | 1200 | removeTopic(topicName); 1201 | ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); 1202 | 1203 | unsubscribeFuture.complete(null); 1204 | log.info("[{}] [{}] [{}] Unsubscribed Topics Consumer, allTopicPartitionsNumber: {}", 1205 | topicName, subscription, consumerName, allTopicPartitionsNumber); 1206 | } else { 1207 | unsubscribeFuture.completeExceptionally(ex); 1208 | setState(State.Failed); 1209 | log.error("[{}] [{}] [{}] Could not unsubscribe Topics Consumer", 1210 | topicName, subscription, consumerName, ex.getCause()); 1211 | } 1212 | }); 1213 | 1214 | return unsubscribeFuture; 1215 | } 1216 | 1217 | // Remove a consumer for a topic 1218 | public CompletableFuture removeConsumerAsync(String topicName) { 1219 | checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); 1220 | 1221 | if (getState() == State.Closing || getState() == State.Closed) { 1222 | return FutureUtil.failedFuture( 1223 | new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); 1224 | } 1225 | 1226 | CompletableFuture unsubscribeFuture = new CompletableFuture<>(); 1227 | String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); 1228 | 1229 | 1230 | List> consumersToClose = consumers.values().stream() 1231 | .filter(consumer -> { 1232 | String consumerTopicName = consumer.getTopic(); 1233 | return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); 1234 | }).collect(Collectors.toList()); 1235 | 1236 | List> futureList = consumersToClose.stream() 1237 | .map(ConsumerImpl::closeAsync).collect(Collectors.toList()); 1238 | 1239 | FutureUtil.waitForAll(futureList) 1240 | .whenComplete((r, ex) -> { 1241 | if (ex == null) { 1242 | consumersToClose.forEach(consumer1 -> { 1243 | consumers.remove(consumer1.getTopic()); 1244 | pausedConsumers.remove(consumer1); 1245 | allTopicPartitionsNumber.decrementAndGet(); 1246 | }); 1247 | 1248 | removeTopic(topicName); 1249 | ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); 1250 | 1251 | unsubscribeFuture.complete(null); 1252 | log.info("[{}] [{}] [{}] Removed Topics Consumer, allTopicPartitionsNumber: {}", 1253 | topicName, subscription, consumerName, allTopicPartitionsNumber); 1254 | } else { 1255 | unsubscribeFuture.completeExceptionally(ex); 1256 | setState(State.Failed); 1257 | log.error("[{}] [{}] [{}] Could not remove Topics Consumer", 1258 | topicName, subscription, consumerName, ex.getCause()); 1259 | } 1260 | }); 1261 | 1262 | return unsubscribeFuture; 1263 | } 1264 | 1265 | // get topics name 1266 | public List getPartitionedTopics() { 1267 | return partitionedTopics.keySet().stream().collect(Collectors.toList()); 1268 | } 1269 | 1270 | // get partitioned topics name 1271 | public List getPartitions() { 1272 | return consumers.keySet().stream().collect(Collectors.toList()); 1273 | } 1274 | 1275 | // get partitioned consumers 1276 | public List> getConsumers() { 1277 | return consumers.values().stream().collect(Collectors.toList()); 1278 | } 1279 | 1280 | // get all partitions that in the topics map 1281 | int getPartitionsOfTheTopicMap() { 1282 | return partitionedTopics.values().stream().mapToInt(Integer::intValue).sum(); 1283 | } 1284 | 1285 | @Override 1286 | public void pause() { 1287 | synchronized (pauseMutex) { 1288 | paused = true; 1289 | consumers.forEach((name, consumer) -> consumer.pause()); 1290 | } 1291 | } 1292 | 1293 | @Override 1294 | public void resume() { 1295 | synchronized (pauseMutex) { 1296 | paused = false; 1297 | consumers.forEach((name, consumer) -> consumer.resume()); 1298 | } 1299 | } 1300 | 1301 | @Override 1302 | public long getLastDisconnectedTimestamp() { 1303 | long lastDisconnectedTimestamp = 0; 1304 | Optional> c = consumers.values().stream().max(Comparator.comparingLong(ConsumerImpl::getLastDisconnectedTimestamp)); 1305 | if (c.isPresent()) { 1306 | lastDisconnectedTimestamp = c.get().getLastDisconnectedTimestamp(); 1307 | } 1308 | return lastDisconnectedTimestamp; 1309 | } 1310 | 1311 | // subscribe increased partitions for a given topic 1312 | private CompletableFuture subscribeIncreasedTopicPartitions(String topicName) { 1313 | int oldPartitionNumber = partitionedTopics.get(topicName); 1314 | 1315 | return client.getPartitionsForTopic(topicName).thenCompose(list -> { 1316 | int currentPartitionNumber = Long.valueOf(list.stream().filter(t -> TopicName.get(t).isPartitioned()).count()).intValue(); 1317 | 1318 | if (log.isDebugEnabled()) { 1319 | log.debug("[{}] partitions number. old: {}, new: {}", 1320 | topicName, oldPartitionNumber, currentPartitionNumber); 1321 | } 1322 | 1323 | if (oldPartitionNumber == currentPartitionNumber) { 1324 | // topic partition number not changed 1325 | return CompletableFuture.completedFuture(null); 1326 | } else if (currentPartitionNumber == PartitionedTopicMetadata.NON_PARTITIONED) { 1327 | // The topic was initially partitioned but then it was deleted. We keep it in the topics 1328 | partitionedTopics.put(topicName, 0); 1329 | 1330 | allTopicPartitionsNumber.addAndGet(-oldPartitionNumber); 1331 | List> futures = new ArrayList<>(); 1332 | for (Iterator>> it = consumers.entrySet().iterator(); it.hasNext(); ) { 1333 | Map.Entry> e = it.next(); 1334 | String partitionedTopicName = TopicName.get(e.getKey()).getPartitionedTopicName(); 1335 | 1336 | // Remove the consumers that belong to the deleted partitioned topic 1337 | if (partitionedTopicName.equals(topicName)) { 1338 | futures.add(e.getValue().closeAsync()); 1339 | consumers.remove(e.getKey()); 1340 | } 1341 | } 1342 | 1343 | return FutureUtil.waitForAll(futures); 1344 | } else if (oldPartitionNumber < currentPartitionNumber) { 1345 | allTopicPartitionsNumber.addAndGet(currentPartitionNumber - oldPartitionNumber); 1346 | partitionedTopics.put(topicName, currentPartitionNumber); 1347 | List newPartitions = list.subList(oldPartitionNumber, currentPartitionNumber); 1348 | // subscribe new added partitions 1349 | List>> futureList = newPartitions 1350 | .stream() 1351 | .map(partitionName -> { 1352 | int partitionIndex = TopicName.getPartitionIndex(partitionName); 1353 | CompletableFuture> subFuture = new CompletableFuture<>(); 1354 | ConsumerConfigurationData configurationData = getInternalConsumerConfig(); 1355 | int receiverQueueSize = getReceiverQueueSize(topicName); 1356 | configurationData.setReceiverQueueSize(receiverQueueSize); 1357 | ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl( 1358 | client, partitionName, configurationData, 1359 | client.externalExecutorProvider(), 1360 | partitionIndex, true, subFuture, null, schema, interceptors, 1361 | true /* createTopicIfDoesNotExist */); 1362 | synchronized (pauseMutex) { 1363 | if (paused) { 1364 | newConsumer.pause(); 1365 | } 1366 | addMessageCounterForTopic(newConsumer.getTopic()); 1367 | consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); 1368 | } 1369 | if (log.isDebugEnabled()) { 1370 | log.debug("[{}] create consumer {} for partitionName: {}", 1371 | topicName, newConsumer.getTopic(), partitionName); 1372 | } 1373 | return subFuture; 1374 | }) 1375 | .collect(Collectors.toList()); 1376 | 1377 | // wait for all partitions subscribe future complete, then startReceivingMessages 1378 | return FutureUtil.waitForAll(futureList) 1379 | .thenAccept(finalFuture -> { 1380 | List> newConsumerList = newPartitions.stream() 1381 | .map(partitionTopic -> consumers.get(partitionTopic)) 1382 | .collect(Collectors.toList()); 1383 | startReceivingMessages(newConsumerList); 1384 | }); 1385 | } else { 1386 | log.error("[{}] not support shrink topic partitions. old: {}, new: {}", 1387 | topicName, oldPartitionNumber, currentPartitionNumber); 1388 | return FutureUtil.failedFuture(new NotSupportedException("not support shrink topic partitions")); 1389 | } 1390 | }); 1391 | } 1392 | 1393 | private final TimerTask partitionsAutoUpdateTimerTask = new TimerTask() { 1394 | @Override 1395 | public void run(Timeout timeout) throws Exception { 1396 | if (timeout.isCancelled() || getState() != State.Ready) { 1397 | return; 1398 | } 1399 | 1400 | if (log.isDebugEnabled()) { 1401 | log.debug("[{}] run partitionsAutoUpdateTimerTask", topic); 1402 | } 1403 | 1404 | // if last auto update not completed yet, do nothing. 1405 | if (partitionsAutoUpdateFuture == null || partitionsAutoUpdateFuture.isDone()) { 1406 | partitionsAutoUpdateFuture = topicsPartitionChangedListener.onTopicsExtended(partitionedTopics.keySet()); 1407 | } 1408 | 1409 | // schedule the next re-check task 1410 | partitionsAutoUpdateTimeout = client.timer() 1411 | .newTimeout(partitionsAutoUpdateTimerTask, conf.getAutoUpdatePartitionsIntervalSeconds(), TimeUnit.SECONDS); 1412 | } 1413 | }; 1414 | 1415 | @VisibleForTesting 1416 | public Timeout getPartitionsAutoUpdateTimeout() { 1417 | return partitionsAutoUpdateTimeout; 1418 | } 1419 | 1420 | @Override 1421 | public CompletableFuture getLastMessageIdAsync() { 1422 | CompletableFuture returnFuture = new CompletableFuture<>(); 1423 | 1424 | Map> messageIdFutures = consumers.entrySet().stream() 1425 | .map(entry -> Pair.of(entry.getKey(), entry.getValue().getLastMessageIdAsync())) 1426 | .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); 1427 | 1428 | CompletableFuture 1429 | .allOf(messageIdFutures.entrySet().stream().map(Map.Entry::getValue).toArray(CompletableFuture[]::new)) 1430 | .whenComplete((ignore, ex) -> { 1431 | ImmutableMap.Builder builder = ImmutableMap.builder(); 1432 | messageIdFutures.forEach((key, future) -> { 1433 | MessageId messageId; 1434 | try { 1435 | messageId = future.get(); 1436 | } catch (Exception e) { 1437 | log.warn("[{}] Exception when topic {} getLastMessageId.", key, e); 1438 | messageId = MessageId.earliest; 1439 | } 1440 | builder.put(key, messageId); 1441 | }); 1442 | returnFuture.complete(new MultiMessageIdImpl(builder.build())); 1443 | }); 1444 | 1445 | return returnFuture; 1446 | } 1447 | 1448 | public void tryAcknowledgeMessage(Message msg) { 1449 | if (msg != null) { 1450 | acknowledgeCumulativeAsync(msg); 1451 | } 1452 | } 1453 | 1454 | // This listener is triggered when topics partitions are updated. 1455 | private class TopicsPartitionChangedListener implements PartitionsChangedListener { 1456 | // Check partitions changes of passed in topics, and subscribe new added partitions. 1457 | @Override 1458 | public CompletableFuture onTopicsExtended(Collection topicsExtended) { 1459 | CompletableFuture future = new CompletableFuture<>(); 1460 | if (topicsExtended.isEmpty()) { 1461 | future.complete(null); 1462 | return future; 1463 | } 1464 | 1465 | if (log.isDebugEnabled()) { 1466 | log.debug("[{}] run onTopicsExtended: {}, size: {}", 1467 | topic, topicsExtended, topicsExtended.size()); 1468 | } 1469 | 1470 | List> futureList = Lists.newArrayListWithExpectedSize(topicsExtended.size()); 1471 | topicsExtended.forEach(topic -> futureList.add(subscribeIncreasedTopicPartitions(topic))); 1472 | FutureUtil.waitForAll(futureList) 1473 | .thenAccept(finalFuture -> future.complete(null)) 1474 | .exceptionally(ex -> { 1475 | log.warn("[{}] Failed to subscribe increased topics partitions: {}", topic, ex.getMessage()); 1476 | future.completeExceptionally(ex); 1477 | return null; 1478 | }); 1479 | 1480 | return future; 1481 | } 1482 | } 1483 | 1484 | 1485 | } 1486 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/ExponentialWeightDistribution.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | public class ExponentialWeightDistribution extends RangeDistribution { 19 | 20 | public ExponentialWeightDistribution(int lowerBound, int upperBound, int points) { 21 | super(lowerBound, upperBound, points); 22 | } 23 | 24 | @Override 25 | protected int[] distribute(int lowerBound, int upperBound, int points) { 26 | if (points == 1) { 27 | return new int[]{lowerBound}; 28 | } else if (points == 2) { 29 | return new int[]{lowerBound, upperBound}; 30 | } 31 | 32 | int[] distribution = new int[points]; 33 | double multiplier = Math.pow((upperBound * 1.0) / lowerBound, 1.0 / (points - 1)); 34 | for (int i = 0; i < points; i++) { 35 | if (i == points - 1) { 36 | distribution[i] = upperBound; 37 | } else { 38 | distribution[i] = (int) Math.round(lowerBound * Math.pow(multiplier, i)); 39 | } 40 | } 41 | return distribution; 42 | } 43 | 44 | @Override 45 | public String getName() { 46 | return "EXPONENTIAL"; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/LinearWeightDistribution.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | public class LinearWeightDistribution extends RangeDistribution { 19 | 20 | public LinearWeightDistribution(int lowerBound, int upperBound, int points) { 21 | super(lowerBound, upperBound, points); 22 | } 23 | 24 | @Override 25 | protected int[] distribute(int lowerBound, int upperBound, int points) { 26 | int[] distribution = new int[points]; 27 | for (int i = 0; i < points; i++) { 28 | distribution[i] = lowerBound + Math.round(i * (upperBound - lowerBound) / ((points - 1) * 1.0f)); 29 | } 30 | return distribution; 31 | } 32 | 33 | @Override 34 | public String getName() { 35 | return "LINEAR"; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/RangeDistribution.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | import org.slf4j.Logger; 19 | import org.slf4j.LoggerFactory; 20 | 21 | public abstract class RangeDistribution implements WeightDistribution { 22 | private static final Logger log = LoggerFactory.getLogger(RangeDistribution.class); 23 | 24 | private final int[] distribution; 25 | 26 | public RangeDistribution(int lowerBound, int upperBound, int maxWeight) { 27 | validate(lowerBound, upperBound, maxWeight); 28 | 29 | distribution = distribute(lowerBound, upperBound, maxWeight); 30 | if (distribution == null) { 31 | throw new IllegalArgumentException("Null distribution generated in the strategy=" + getName()); 32 | } 33 | 34 | log.info("Distribution for strategy={}: {}", getName(), distributionToString()); 35 | } 36 | 37 | public int getValue(int weight) { 38 | if (weight <= 0 || weight > distribution.length) { 39 | throw new IllegalArgumentException("Invalid weight=" + weight + " supplied"); 40 | } 41 | return distribution[weight -1]; 42 | } 43 | 44 | public int getMinValue() { 45 | return distribution[0]; 46 | } 47 | 48 | public int getMaxValue() { 49 | return distribution[distribution.length - 1]; 50 | } 51 | 52 | @Override 53 | public String toString() { 54 | return String.format("name=%s, distribution=%s", getName(), distributionToString()); 55 | } 56 | 57 | protected abstract int[] distribute(int lowerBound, int upperBound, int points); 58 | 59 | protected void validate(int lowerBound, int upperBound, int points) { 60 | if (lowerBound < 1) { 61 | throw new IllegalArgumentException("Min for lower bound is 1, provided=" + lowerBound); 62 | } 63 | if (points <= 0) { 64 | throw new IllegalArgumentException("Points should be greater than zero, provided=" + points); 65 | } 66 | if (upperBound < lowerBound) { 67 | throw new IllegalArgumentException(String.format("Upper bound should be greater than or equal to lower bound=%d, provided=%d", 68 | lowerBound, upperBound)); 69 | } 70 | } 71 | 72 | private String distributionToString() { 73 | StringBuilder builder = new StringBuilder(); 74 | for (int i = 0; i < distribution.length; i++) { 75 | if (i > 0) { 76 | builder.append(", "); 77 | } 78 | builder.append(distribution[i]); 79 | } 80 | return builder.toString(); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/TopicMessageCountTracker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | import org.apache.pulsar.client.api.Message; 19 | 20 | import java.util.Collection; 21 | import java.util.Collections; 22 | import java.util.Iterator; 23 | import java.util.Map; 24 | import java.util.concurrent.ConcurrentHashMap; 25 | import java.util.concurrent.atomic.AtomicInteger; 26 | import java.util.stream.Collectors; 27 | 28 | /** 29 | * Specialized collection used when clearing incoming message queue in a multi consumer is required 30 | * Instead, the queue is drained to this collection which tracks the count of messages for every topic 31 | */ 32 | public class TopicMessageCountTracker implements Collection { 33 | Map counters = new ConcurrentHashMap<>(); 34 | 35 | public Map getCounters() { 36 | Map ret = counters.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue().get())); 37 | return Collections.unmodifiableMap(ret); 38 | } 39 | 40 | public boolean add(Message m) { 41 | counters.computeIfAbsent(m.getTopicName(), (k -> new AtomicInteger())); 42 | counters.get(m.getTopicName()).incrementAndGet(); 43 | return true; 44 | } 45 | 46 | public boolean addAll(Collection c) { 47 | c.stream().forEach(m -> add(m)); 48 | return true; 49 | } 50 | 51 | public int size() { 52 | throw new UnsupportedOperationException(); 53 | } 54 | 55 | public boolean isEmpty() { 56 | throw new UnsupportedOperationException(); 57 | } 58 | 59 | public boolean contains(Object o) { 60 | throw new UnsupportedOperationException(); 61 | } 62 | 63 | public Iterator iterator() { 64 | throw new UnsupportedOperationException(); 65 | } 66 | 67 | public Object[] toArray() { 68 | throw new UnsupportedOperationException(); 69 | } 70 | 71 | public T[] toArray(T[] a) { 72 | throw new UnsupportedOperationException(); 73 | } 74 | 75 | public boolean remove(Object o) { 76 | throw new UnsupportedOperationException(); 77 | } 78 | 79 | public boolean containsAll(Collection c) { 80 | throw new UnsupportedOperationException(); 81 | } 82 | 83 | public boolean removeAll(Collection c) { 84 | throw new UnsupportedOperationException(); 85 | } 86 | 87 | public boolean retainAll(Collection c) { 88 | throw new UnsupportedOperationException(); 89 | } 90 | 91 | public void clear() { 92 | counters.clear(); 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/TopicThresholdDistribution.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | public interface TopicThresholdDistribution { 19 | int getWeight(String topic); 20 | int getMinValue(); 21 | int getMaxValue(); 22 | int getValue(String topic); 23 | } 24 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/TopicThresholdDistributionImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | import org.apache.pulsar.common.naming.TopicName; 19 | import org.slf4j.Logger; 20 | import org.slf4j.LoggerFactory; 21 | 22 | import java.util.Map; 23 | 24 | public class TopicThresholdDistributionImpl implements TopicThresholdDistribution { 25 | private static final Logger log = LoggerFactory.getLogger(TopicThresholdDistributionImpl.class); 26 | private final WeightDistribution distribution; 27 | private final Map topicWeights; 28 | 29 | private TopicThresholdDistributionImpl(Map topicWeights, WeightDistribution distribution) { 30 | this.topicWeights = topicWeights; 31 | this.distribution = distribution; 32 | } 33 | 34 | public int getWeight(String topic) { 35 | TopicName topicName = TopicName.get(topic); 36 | Integer weight = topicWeights.get(topic); 37 | 38 | if (weight == null && topicName.isPartitioned()) { 39 | weight = topicWeights.get(topicName.getPartitionedTopicName()); 40 | } 41 | if (weight == null) { 42 | log.warn("Weight not found for topic={}, default to weight=1", topic); 43 | weight = 1; 44 | } 45 | return weight; 46 | } 47 | 48 | public int getMinValue() { 49 | return distribution.getMinValue(); 50 | } 51 | 52 | public int getMaxValue() { 53 | return distribution.getMaxValue(); 54 | } 55 | 56 | public int getValue(String topic) { 57 | int bound = distribution.getValue(getWeight(topic)); 58 | log.debug("Topic threshold for topic={} is {}", topic, bound); 59 | return bound; 60 | } 61 | 62 | public static TopicThresholdDistribution loadFromConf(WeightedConsumerConfiguration conf) 63 | throws IllegalArgumentException { 64 | int maxWeight = 1; 65 | for (Map.Entry entry : conf.getTopicWeights().entrySet()) { 66 | maxWeight = Math.max(maxWeight, entry.getValue()); 67 | } 68 | WeightDistribution weightDistribution; 69 | switch (conf.getDistributionStrategy()) { 70 | case LINEAR: 71 | weightDistribution = new LinearWeightDistribution(conf.getMinBound(), conf.getMaxBound(), maxWeight); 72 | break; 73 | case EXPONENTIAL: 74 | weightDistribution = new ExponentialWeightDistribution(conf.getMinBound(), conf.getMaxBound(), maxWeight); 75 | break; 76 | default: 77 | throw new IllegalArgumentException("Weight strategy " + conf.getDistributionStrategy() + " is not supported"); 78 | } 79 | 80 | return new TopicThresholdDistributionImpl(conf.getTopicWeights(), weightDistribution); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/WeightDistribution.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | public interface WeightDistribution { 19 | String getName(); 20 | int getValue(int weight); 21 | int getMinValue(); 22 | int getMaxValue(); 23 | } 24 | -------------------------------------------------------------------------------- /consumer-java/src/main/java/org/apache/pulsar/client/impl/weight/WeightedConsumerConfiguration.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright 2022-2026, the original author or authors. 4 | * * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * * you may not use this file except in compliance with the License. 6 | * * You may obtain a copy of the License at 7 | * * http://www.apache.org/licenses/LICENSE-2.0 8 | * * Unless required by applicable law or agreed to in writing, software 9 | * * distributed under the License is distributed on an "AS IS" BASIS, 10 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * * See the License for the specific language governing permissions and 12 | * * limitations under the License. 13 | * 14 | */ 15 | 16 | package org.apache.pulsar.client.impl.weight; 17 | 18 | 19 | import org.apache.pulsar.client.api.ConsumerBuilder; 20 | import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; 21 | import org.apache.pulsar.common.naming.TopicName; 22 | import org.apache.pulsar.shade.com.google.common.base.Preconditions; 23 | 24 | import java.util.Collections; 25 | import java.util.HashMap; 26 | import java.util.Map; 27 | import java.util.SortedMap; 28 | 29 | public class WeightedConsumerConfiguration { 30 | public static String WT_CONF_PREFIX = "WT_"; 31 | //Different prefix for topics to prevent conflict, refer to #populateBuilder for usage 32 | public static String WT_TOPIC_CONF_PREFIX = "WTP_"; 33 | 34 | private DistributionStrategy distributionStrategy = DistributionStrategy.LINEAR; 35 | private int minBound = 100; 36 | private int maxBound = 1000; 37 | private Map topicWeights = new HashMap<>(); 38 | private boolean throttleReceiveQueue = false; 39 | private int queueResumeThreshold = 0; 40 | private int maxWeightAllowed = 100; 41 | 42 | public void setDistributionStrategy(DistributionStrategy distributionStrategy) { 43 | this.distributionStrategy = distributionStrategy; 44 | } 45 | 46 | public void setMinBound(int minBound) { 47 | this.minBound = minBound; 48 | } 49 | 50 | public void setMaxBound(int maxBound) { 51 | this.maxBound = maxBound; 52 | } 53 | 54 | public void addTopic(String topic, Integer weight) { 55 | Preconditions.checkNotNull(topic); 56 | if(weight == null) { 57 | weight = parseWeightFromTopic(topic); 58 | if(weight == null) { 59 | weight = 1; 60 | } 61 | } 62 | this.topicWeights.put(topic, weight); 63 | } 64 | 65 | /** 66 | * In situations where local message processing is near instantaneous, user will not see any weighted consumption 67 | * across different topics irrespective of the weights assigned. In a way this can be desired behaviour to not 68 | * penalize low weighted topics if their message processing is fast enough (or faster than the network fetch). 69 | * 70 | * But in specific circumstances, you might want to enforce consumption to happen in a weighted manner and 71 | * artificially limit the messages to be polled from lower weighted topics. This flag does that at the cost of loss 72 | * of capability to make optimal use of local resources for maximum throughput. 73 | * 74 | * Set this to true only if you are clear with the impact of the flag 75 | * 76 | * @param shouldThrottle 77 | */ 78 | public void setThrottleReceiveQueue(boolean shouldThrottle) { 79 | this.throttleReceiveQueue = shouldThrottle; 80 | } 81 | 82 | /** 83 | * Caution! One should never need to change this conf in normal circumstances. 84 | * Default queue resume threshold is zero which should provide the fairest weighted consumption 85 | * Changing this threshold to a positive value can result in lower weighted consumers not adhering to the distribution 86 | * Made configurable for debugging purposes 87 | * Applied bounds are [0, {@link #minBound}] 88 | * 89 | * @return 90 | */ 91 | public void setQueueResumeThreshold(int queueResumeThreshold) { 92 | this.queueResumeThreshold = queueResumeThreshold; 93 | } 94 | 95 | public void setMaxWeightAllowed(int maxWeight) { 96 | this.maxWeightAllowed = maxWeight; 97 | } 98 | 99 | public DistributionStrategy getDistributionStrategy() { 100 | return distributionStrategy; 101 | } 102 | 103 | public int getMinBound() { 104 | return minBound; 105 | } 106 | 107 | public int getMaxBound() { 108 | return maxBound; 109 | } 110 | 111 | public Map getTopicWeights() { 112 | return Collections.unmodifiableMap(topicWeights); 113 | } 114 | 115 | public boolean isThrottleReceiveQueue() { 116 | return throttleReceiveQueue; 117 | } 118 | 119 | public int getQueueResumeThreshold() { 120 | return queueResumeThreshold; 121 | } 122 | 123 | public int getMaxWeightAllowed() { 124 | return maxWeightAllowed; 125 | } 126 | 127 | @Override 128 | public String toString() { 129 | StringBuilder str = new StringBuilder(); 130 | str.append("Weight configuration:"); 131 | str.append(" dist_strategy=").append(distributionStrategy); 132 | str.append(" min_bound=").append(minBound); 133 | str.append(" max_bound=").append(maxBound); 134 | str.append(" max_weight_allowed=").append(maxWeightAllowed); 135 | str.append(" throttle_recv_queue=").append(throttleReceiveQueue); 136 | str.append(" queue_resume_threshold=").append(queueResumeThreshold); 137 | str.append(" topic_weights=["); 138 | topicWeights.forEach((topic, weight) -> str.append(" {").append(topic).append(", ").append(weight).append("}")); 139 | str.append(" ]"); 140 | return str.toString(); 141 | } 142 | 143 | private void validate() { 144 | Preconditions.checkArgument(maxWeightAllowed >= 1, "max weight allowed should be equal or more than 1"); 145 | Preconditions.checkArgument(minBound >= 100, "min bound should be at least 100"); 146 | Preconditions.checkArgument(maxBound >= minBound, "max bound should be at least as much as min bound %s", minBound); 147 | Preconditions.checkArgument(queueResumeThreshold >= 0 && queueResumeThreshold <= minBound, "queue resume threshold should be in the range [0,minBound(%s)]", minBound); 148 | if(topicWeights.size() > 0) { 149 | for (Map.Entry entry : topicWeights.entrySet()) { 150 | Integer weight = entry.getValue(); 151 | Preconditions.checkNotNull(weight); 152 | Preconditions.checkArgument(weight >= 1 && weight <= maxWeightAllowed, 153 | "non-null weights should be in the range [1,maxWeightAllowed(%s)], found %s for topic %s", 154 | maxWeightAllowed, weight.toString(), entry.getKey()); 155 | } 156 | } 157 | } 158 | 159 | public void populateBuilder(ConsumerBuilder builder) { 160 | validate(); 161 | addProp(builder, "DIST_STRATEGY", distributionStrategy); 162 | addProp(builder, "MIN_BOUND", minBound); 163 | addProp(builder, "MAX_BOUND", maxBound); 164 | addProp(builder, "THROTTLE_RQ", throttleReceiveQueue); 165 | addProp(builder, "RESUME_THRESHOLD", queueResumeThreshold); 166 | addProp(builder, "MAX_WT_ALLOWED", maxWeightAllowed); 167 | topicWeights.forEach((topic, weight) -> populateTopicInBuilder(builder, topic, weight)); 168 | } 169 | 170 | public static void populateTopicInBuilder(ConsumerBuilder builder, String topic, Integer weight) { 171 | builder.topic(topic); 172 | builder.property(WT_TOPIC_CONF_PREFIX + topic, weight == null ? "" : weight.toString()); 173 | } 174 | 175 | public static WeightedConsumerConfiguration loadFromConf(ConsumerConfigurationData conf) { 176 | WeightedConsumerConfiguration weightConf = new WeightedConsumerConfiguration(); 177 | weightConf.setDistributionStrategy(DistributionStrategy.valueOf(readPropOrDefault(conf, "DIST_STRATEGY", DistributionStrategy.LINEAR.name()))); 178 | weightConf.setMinBound(parseConfAsIntOrDefault(conf, "MIN_BOUND", 100)); 179 | weightConf.setMaxBound(parseConfAsIntOrDefault(conf, "MAX_BOUND", 1000)); 180 | weightConf.setThrottleReceiveQueue(Boolean.parseBoolean(readPropOrDefault(conf, "THROTTLE_RQ", "false"))); 181 | weightConf.setQueueResumeThreshold(parseConfAsIntOrDefault(conf, "RESUME_THRESHOLD", 0)); 182 | weightConf.setMaxWeightAllowed(parseConfAsIntOrDefault(conf, "MAX_WT_ALLOWED", 100)); 183 | 184 | SortedMap props = conf.getProperties(); 185 | for (String confKey : props.tailMap(WT_TOPIC_CONF_PREFIX).keySet()) { 186 | if (confKey.startsWith(WT_TOPIC_CONF_PREFIX)) { 187 | String topic = confKey.split(WT_TOPIC_CONF_PREFIX)[1]; 188 | Integer weight; 189 | try { 190 | weight = Integer.parseInt(conf.getProperties().get(confKey)); 191 | } catch (NumberFormatException ex) { 192 | weight = null; 193 | } 194 | weightConf.addTopic(topic, weight); 195 | } 196 | } 197 | 198 | weightConf.validate(); 199 | return weightConf; 200 | } 201 | 202 | private static void addProp(ConsumerBuilder builder, String key, Object val) { 203 | builder.property(WT_CONF_PREFIX + key, val.toString()); 204 | } 205 | 206 | private static String readPropOrDefault(ConsumerConfigurationData conf, String key, String defaultVal) { 207 | String val = conf.getProperties().get(WT_CONF_PREFIX + key); 208 | return val != null ? val : defaultVal; 209 | } 210 | 211 | private static Integer parseConfAsIntOrDefault(ConsumerConfigurationData conf, String key, Integer defaultVal) { 212 | try { 213 | return Integer.parseInt(readPropOrDefault(conf, key, defaultVal != null ? defaultVal.toString() : null)); 214 | } catch (NumberFormatException ex) { 215 | return defaultVal; 216 | } 217 | } 218 | 219 | /** 220 | * Returns weight if the topic(partitioned or otherwise) follows the convention of appending weight to the name 221 | * Returns null if weight cannot be determined 222 | * @param topic 223 | * @return 224 | */ 225 | private static Integer parseWeightFromTopic(String topic) { 226 | TopicName topicName = TopicName.get(topic); 227 | if (topicName.isPartitioned()) { 228 | topic = topicName.getPartitionedTopicName(); 229 | } 230 | String[] parts = topic.split("-weight-"); 231 | if (parts.length == 2) { 232 | try { 233 | return Integer.valueOf(parts[1]); 234 | } catch (NumberFormatException ex) { 235 | } //ignored 236 | } 237 | return null; 238 | } 239 | 240 | public enum DistributionStrategy { 241 | LINEAR, EXPONENTIAL 242 | } 243 | } 244 | -------------------------------------------------------------------------------- /docs/images/lagging_backlog.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/flipkart-incubator/pulsar-weighted-consumer/dcc10bbed7d827eff700fd10e7096f443bffd2d5/docs/images/lagging_backlog.png -------------------------------------------------------------------------------- /docs/images/lagging_thpt.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/flipkart-incubator/pulsar-weighted-consumer/dcc10bbed7d827eff700fd10e7096f443bffd2d5/docs/images/lagging_thpt.png -------------------------------------------------------------------------------- /docs/images/tailing_thpt.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/flipkart-incubator/pulsar-weighted-consumer/dcc10bbed7d827eff700fd10e7096f443bffd2d5/docs/images/tailing_thpt.png -------------------------------------------------------------------------------- /examples/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.flipkart 8 | pulsar-weighted-consumer-examples 9 | 2.10-SNAPSHOT 10 | 11 | 12 | 8 13 | 2.10.0 14 | 15 | 16 | 17 | 18 | 19 | org.apache.maven.plugins 20 | maven-compiler-plugin 21 | 22 | ${java.version} 23 | ${java.version} 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | org.apache.pulsar 32 | pulsar-client 33 | ${pulsar.version} 34 | 35 | 36 | com.flipkart 37 | pulsar-weighted-consumer 38 | 2.10-0 39 | 40 | 41 | org.apache.logging.log4j 42 | log4j-slf4j-impl 43 | 2.11.1 44 | 45 | 46 | 47 | -------------------------------------------------------------------------------- /examples/src/main/java/examples/StatPrinter.java: -------------------------------------------------------------------------------- 1 | package examples; 2 | 3 | import org.apache.pulsar.client.api.Consumer; 4 | import org.apache.pulsar.client.api.ConsumerStats; 5 | import org.apache.pulsar.client.impl.ConsumerImpl; 6 | import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; 7 | 8 | import java.text.DecimalFormat; 9 | import java.time.Duration; 10 | import java.util.List; 11 | 12 | public class StatPrinter { 13 | public static DecimalFormat THROUGHPUT_FORMAT = new DecimalFormat("0.00"); 14 | public static int statIntervalSecs = 5; 15 | 16 | public static void print(Consumer consumer) { 17 | try { 18 | //Stats are calculated at fixed intervals in pulsar clients. Waiting to ensure latest aggregation is completed. 19 | Thread.sleep(Duration.ofSeconds(statIntervalSecs + 5).toMillis()); 20 | if (consumer instanceof ConsumerImpl) { 21 | print((ConsumerImpl) consumer); 22 | } else if (consumer instanceof MultiTopicsConsumerImpl) { 23 | print((MultiTopicsConsumerImpl) consumer); 24 | } 25 | } catch (InterruptedException ex) { 26 | //swallow and exit 27 | } 28 | } 29 | 30 | private static void print(ConsumerImpl consumer) { 31 | System.out.println("Single Consumer stats\n-----"); 32 | System.out.printf("[%s] [%s] [%s]%n", 33 | consumer.getTopic(), consumer.getSubscription(), consumer.getConsumerName()); 34 | print(consumer.getStats()); 35 | } 36 | 37 | private static void print(MultiTopicsConsumerImpl consumer) { 38 | System.out.println("Multi Consumer stats\n-----"); 39 | System.out.printf("Internal-consumers=[%s] [%s] [%s]%n", 40 | consumer.getConsumers().size(), consumer.getSubscription(), consumer.getConsumerName()); 41 | print(consumer.getStats()); 42 | List> consumers = consumer.getConsumers(); 43 | consumers.forEach(StatPrinter::print); 44 | } 45 | 46 | private static void print(ConsumerStats stats) { 47 | System.out.printf("Consume Total: %d msgs --- %d bytes --- Pending prefetched messages: %d%n", 48 | stats.getTotalMsgsReceived(), stats.getTotalBytesReceived(), stats.getMsgNumInReceiverQueue()); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /examples/src/main/java/examples/UnorderedConsumptionJob.java: -------------------------------------------------------------------------------- 1 | package examples; 2 | 3 | import org.apache.pulsar.client.api.*; 4 | import org.apache.pulsar.client.api.url.DataURLStreamHandler; 5 | import org.apache.pulsar.client.impl.WeightedConsumerBuilder; 6 | import org.apache.pulsar.client.impl.auth.oauth2.AuthenticationFactoryOAuth2; 7 | import org.apache.pulsar.client.impl.weight.WeightedConsumerConfiguration; 8 | import org.apache.pulsar.shade.com.google.gson.Gson; 9 | 10 | import java.net.MalformedURLException; 11 | import java.net.URL; 12 | import java.util.*; 13 | import java.util.concurrent.*; 14 | import java.util.concurrent.atomic.AtomicBoolean; 15 | import java.util.concurrent.atomic.AtomicLong; 16 | 17 | public class UnorderedConsumptionJob implements Runnable { 18 | public static void main(String[] args) { 19 | UnorderedConsumptionJob job = new UnorderedConsumptionJob(4, 3600); 20 | job.run(); 21 | } 22 | 23 | /** 24 | * Configure number of workers you want for processing messages 25 | * 26 | * UNORDERED: Parallelism will be determined by this knob and can be greater than number of partitions 27 | * 28 | * ORDERED: If multiple consumers are run with same subscription, they will divide partitions among themselves 29 | * You can simulate this by running multiple instances of ConsumeDriver concurrently. When you are looking for parallel 30 | * consumption within same consumer, limit number of workers here to max(partitions/consumers, 1) 31 | */ 32 | final int workers; 33 | final int jobDurationSeconds; 34 | final ExecutorService executor; 35 | AtomicBoolean terminator = new AtomicBoolean(false); 36 | 37 | String pulsarUrl; 38 | SubscriptionType subscriptionType; 39 | String subName; 40 | Map topicWeights; 41 | 42 | public UnorderedConsumptionJob(int workers, int jobDurationSeconds) { 43 | this.workers = workers; 44 | this.jobDurationSeconds = jobDurationSeconds; 45 | this.executor = Executors.newFixedThreadPool(workers + 1); 46 | } 47 | 48 | protected void configure() { 49 | pulsarUrl = ""; 50 | subscriptionType = SubscriptionType.Shared; 51 | subName = "sub-a"; 52 | topicWeights = new HashMap() {{ 53 | put("persistent://public/default/ankur-hello-weight-1", null); 54 | put("persistent://public/default/ankur-hello-weight-2", null); 55 | put("persistent://public/default/ankur-hello-weight-3", 5); 56 | }}; 57 | } 58 | 59 | @Override 60 | public void run() { 61 | configure(); 62 | 63 | try { 64 | CompletableFuture job = CompletableFuture.runAsync(() -> { 65 | AtomicLong successCtr = new AtomicLong(0), failureCtr = new AtomicLong(0); 66 | List> tasks = new ArrayList<>(); 67 | 68 | try (PulsarClient client = createPulsarClient(); 69 | Consumer consumer = createPriorityConsumer(client)) { 70 | try { 71 | for (int i = 0; i < workers; i++) { 72 | tasks.add(CompletableFuture.runAsync(() -> { 73 | try { 74 | consumeTask(consumer, successCtr, failureCtr); 75 | } catch (PulsarClientException ex) { 76 | throw new RuntimeException(ex); 77 | } 78 | }, executor)); 79 | } 80 | awaitCompletion(tasks); 81 | } finally { 82 | StatPrinter.print(consumer); 83 | } 84 | } catch (Exception ex) { 85 | throw new RuntimeException(ex); 86 | } finally { 87 | System.out.printf("TOTAL successfully consumed=%d, failures=%d%n", successCtr.get(), failureCtr.get()); 88 | } 89 | }, executor); 90 | 91 | Thread.sleep(jobDurationSeconds * 1000); 92 | terminator.set(true); 93 | job.get(); 94 | } catch (Exception ex) { 95 | throw new RuntimeException(ex); 96 | } finally { 97 | executor.shutdown(); 98 | } 99 | } 100 | 101 | protected PulsarClient createPulsarClient() throws PulsarClientException { 102 | return PulsarClient.builder() 103 | .serviceUrl(pulsarUrl) 104 | .statsInterval(StatPrinter.statIntervalSecs, TimeUnit.SECONDS) 105 | .build(); 106 | } 107 | 108 | protected Consumer createPriorityConsumer(PulsarClient client) throws PulsarClientException { 109 | WeightedConsumerBuilder consumerBuilder = new WeightedConsumerBuilder<>(client, Schema.BYTES) 110 | .distributionStrategy(WeightedConsumerConfiguration.DistributionStrategy.EXPONENTIAL) 111 | .topics(topicWeights); 112 | return consumerBuilder 113 | .subscriptionName(subName) 114 | .subscriptionType(subscriptionType) 115 | .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) 116 | .subscribe(); 117 | } 118 | 119 | protected String processMessage(Message message) throws RuntimeException { 120 | String payload = new String(message.getData()); 121 | try { 122 | Thread.sleep(5); //simulate delay introduced because of actual message processing 123 | } catch (InterruptedException ex) { 124 | throw new RuntimeException(ex); 125 | } 126 | //System.out.println("Processed|" + message.getTopicName() + "|" + message.getMessageId() + "|" + payload); 127 | return payload; 128 | } 129 | 130 | private void consumeTask(Consumer consumer, AtomicLong successCtr, AtomicLong failureCtr) throws PulsarClientException { 131 | long success = 0, failures = 0; 132 | try { 133 | while (!terminator.get()) { 134 | Message message = consumer.receive(1, TimeUnit.SECONDS); 135 | if (message != null) { 136 | try { 137 | processMessage(message); 138 | consumer.acknowledgeAsync(message); 139 | success++; 140 | } catch (Exception ex) { 141 | failures++; 142 | consumer.negativeAcknowledge(message); 143 | } 144 | } 145 | } 146 | } finally { 147 | System.out.printf("[%s] Consumed %d messages successfully and observed %d failures%n", Thread.currentThread().getName(), success, failures); 148 | if (successCtr != null ) { 149 | successCtr.addAndGet(success); 150 | } 151 | if (failureCtr != null) { 152 | failureCtr.addAndGet(failures); 153 | } 154 | } 155 | } 156 | 157 | /** 158 | * Waits for all jobs to finish, throws exception with errors collated across jobs 159 | * Not using {@link CompletableFuture#allOf(CompletableFuture[])} because it throws error as soon as one job fails 160 | */ 161 | private static void awaitCompletion(List> jobs) throws InterruptedException, ExecutionException { 162 | AtomicBoolean errored = new AtomicBoolean(false); 163 | StringBuffer errorBuilder = new StringBuffer("Errors:\n"); 164 | for(CompletableFuture job: jobs) { 165 | job.handle((v, e) -> { 166 | if(e != null) { 167 | errored.set(true); 168 | errorBuilder.append(e.getMessage() + "\n"); 169 | } 170 | return v; 171 | }).get(); 172 | } 173 | if(errored.get()) { 174 | throw new CompletionException(new RuntimeException(errorBuilder.toString())); 175 | } 176 | } 177 | } 178 | -------------------------------------------------------------------------------- /examples/src/main/resources/log4j2.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | --------------------------------------------------------------------------------