From 8071d1577c0c8aeaa3d57269ad00e57176e4aea4 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 21 Dec 2023 15:53:53 -0800 Subject: [PATCH 1/8] PIP-307: Add proxy support --- build/run_unit_group.sh | 2 +- buildtools/pom.xml | 2 +- conf/bookkeeper.conf | 3 + conf/broker.conf | 4 +- conf/proxy.conf | 2 +- conf/standalone.conf | 2 +- .../server/src/assemble/LICENSE.bin.txt | 46 +- .../shell/src/assemble/LICENSE.bin.txt | 44 +- docker/pulsar/Dockerfile | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 3 +- .../mledger/impl/ManagedLedgerTest.java | 55 ++ microbench/README.md | 43 ++ microbench/pom.xml | 133 +++++ .../broker/qos/AsyncTokenBucketBenchmark.java | 81 +++ .../pulsar/broker/qos/package-info.java | 10 +- microbench/src/main/resources/log4j2.xml | 33 ++ pip/README.md | 25 + pip/pip-300.md | 11 +- pip/pip-313.md | 76 +++ pip/pip-318.md | 12 +- pip/pip-320.md | 256 ++++++++++ pip/pip-322.md | 406 +++++++++++++++ pip/pip-323.md | 171 +++++++ pom.xml | 27 +- .../pulsar/broker/ServiceConfiguration.java | 25 +- .../PulsarAuthorizationProvider.java | 3 + .../resources/LoadBalanceResources.java | 38 ++ .../apache/pulsar/broker/PulsarService.java | 12 + .../broker/admin/impl/ClustersBase.java | 36 +- .../broker/admin/impl/NamespacesBase.java | 30 +- .../admin/impl/PersistentTopicsBase.java | 61 --- .../pulsar/broker/admin/v2/Namespaces.java | 3 +- .../pulsar/broker/cache/BundlesQuotas.java | 29 +- .../bucket/BucketDelayedDeliveryTracker.java | 2 + .../broker/loadbalance/NoopLoadManager.java | 2 +- .../extensions/BrokerRegistryImpl.java | 2 +- .../extensions/ExtensibleLoadManagerImpl.java | 120 ++--- .../channel/ServiceUnitStateChannelImpl.java | 33 +- .../extensions/store/LoadDataStore.java | 17 + .../store/TableViewLoadDataStoreImpl.java | 30 +- .../impl/ModularLoadManagerImpl.java | 15 +- .../impl/SimpleLoadManagerImpl.java | 4 +- .../pulsar/broker/lookup/TopicLookupBase.java | 6 +- .../broker/namespace/NamespaceService.java | 66 ++- .../broker/namespace/OwnershipCache.java | 6 +- .../pulsar/broker/qos/AsyncTokenBucket.java | 344 +++++++++++++ .../broker/qos/AsyncTokenBucketBuilder.java | 45 ++ .../qos/DefaultMonotonicSnapshotClock.java | 89 ++++ .../qos/DynamicRateAsyncTokenBucket.java | 68 +++ .../DynamicRateAsyncTokenBucketBuilder.java | 72 +++ .../broker/qos/FinalRateAsyncTokenBucket.java | 64 +++ .../qos/FinalRateAsyncTokenBucketBuilder.java | 62 +++ .../broker/qos/MonotonicSnapshotClock.java | 58 +++ .../pulsar/broker/qos/package-info.java | 22 + .../broker/resourcegroup/ResourceGroup.java | 6 +- .../ResourceGroupPublishLimiter.java | 141 +----- .../resourcegroup/ResourceGroupService.java | 1 - .../service/AbstractBaseDispatcher.java | 56 +-- ...bstractDispatcherSingleActiveConsumer.java | 29 +- .../pulsar/broker/service/AbstractTopic.java | 150 ++---- .../pulsar/broker/service/BrokerService.java | 253 +++------- .../pulsar/broker/service/Consumer.java | 11 +- .../pulsar/broker/service/Dispatcher.java | 18 +- .../broker/service/PrecisePublishLimiter.java | 162 ------ .../pulsar/broker/service/Producer.java | 36 +- .../broker/service/PublishRateLimiter.java | 47 +- .../service/PublishRateLimiterDisable.java | 69 --- .../service/PublishRateLimiterImpl.java | 195 +++++--- .../pulsar/broker/service/ServerCnx.java | 269 +++++----- .../service/ServerCnxThrottleTracker.java | 146 ++++++ .../pulsar/broker/service/Subscription.java | 9 +- .../apache/pulsar/broker/service/Topic.java | 24 +- .../pulsar/broker/service/TransportCnx.java | 28 +- ...PersistentDispatcherMultipleConsumers.java | 15 +- ...sistentDispatcherSingleActiveConsumer.java | 5 - .../NonPersistentSubscription.java | 73 ++- .../nonpersistent/NonPersistentTopic.java | 28 +- .../persistent/DispatchRateLimiter.java | 96 ++-- .../persistent/GeoPersistentReplicator.java | 6 +- .../persistent/MessageDeduplication.java | 21 + ...PersistentDispatcherMultipleConsumers.java | 72 ++- ...sistentDispatcherSingleActiveConsumer.java | 76 ++- .../persistent/PersistentReplicator.java | 8 +- .../persistent/PersistentSubscription.java | 102 ++-- .../service/persistent/PersistentTopic.java | 59 ++- .../service/persistent/ShadowReplicator.java | 2 +- .../persistent/SubscribeRateLimiter.java | 76 +-- .../buffer/impl/InMemTransactionBuffer.java | 5 +- .../buffer/impl/TopicTransactionBuffer.java | 4 +- .../buffer/impl/TransactionBufferDisable.java | 8 +- .../impl/TransactionBufferHandlerImpl.java | 3 +- .../pulsar/broker/web/PulsarWebResource.java | 2 +- .../pulsar/compaction/TwoPhaseCompactor.java | 23 +- .../AdminApiDynamicConfigurationsTest.java | 38 ++ .../broker/admin/PersistentTopicsTest.java | 10 +- .../broker/admin/TopicAutoCreationTest.java | 6 +- .../broker/admin/TopicPoliciesTest.java | 28 +- .../broker/cache/BundlesQuotasTest.java | 18 +- .../SimpleLoadManagerImplTest.java | 58 ++- .../ExtensibleLoadManagerImplTest.java | 473 ++++++++++++------ .../channel/ServiceUnitStateChannelTest.java | 28 +- .../filter/BrokerFilterTestBase.java | 15 + .../scheduler/TransferShedderTest.java | 30 ++ .../extensions/store/LoadDataStoreTest.java | 3 + .../LeastResourceUsageWithWeightTest.java | 15 + .../lookup/http/HttpTopicLookupv2Test.java | 32 ++ .../namespace/NamespaceServiceTest.java | 27 + .../broker/qos/AsyncTokenBucketTest.java | 103 ++++ .../ResourceGroupRateLimiterTest.java | 16 +- .../service/AbstractBaseDispatcherTest.java | 9 +- .../broker/service/AbstractTopicTest.java | 4 + .../BrokerServiceAutoTopicCreationTest.java | 63 +++ .../broker/service/BrokerServiceTest.java | 76 --- .../service/EnableProxyProtocolTest.java | 128 ++++- .../MessagePublishBufferThrottleTest.java | 7 - ...sistentDispatcherFailoverConsumerTest.java | 3 +- .../PersistentTopicInitializeDelayTest.java | 142 ++++++ .../broker/service/PersistentTopicTest.java | 3 +- .../service/PrecisePublishLimiterTest.java | 60 --- .../PublishRateLimiterDisableTest.java | 11 +- .../service/PublishRateLimiterTest.java | 163 +++--- .../service/ReplicatorGlobalNSTest.java | 15 +- .../service/ReplicatorRateLimiterTest.java | 3 + .../service/ReplicatorSubscriptionTest.java | 61 +++ .../pulsar/broker/service/ServerCnxTest.java | 3 +- ...java => TopicPublishRateThrottleTest.java} | 67 +-- .../persistent/MessageDuplicationTest.java | 52 +- .../service/plugin/FilterEntryTest.java | 6 +- .../broker/stats/PrometheusMetricsTest.java | 33 +- .../broker/transaction/TransactionTest.java | 61 ++- .../buffer/TopicTransactionBufferTest.java | 131 +++++ .../buffer/TransactionBufferClientTest.java | 13 +- .../TransactionBufferHandlerImplTest.java | 7 +- .../AuthenticatedProducerConsumerTest.java | 45 ++ .../pulsar/client/api/ClientErrorsTest.java | 2 +- .../api/InjectedClientCnxClientBuilder.java | 52 ++ .../api/MessageDispatchThrottlingTest.java | 149 ++---- ...ListenersWithInternalListenerNameTest.java | 31 +- ...criptionMessageDispatchThrottlingTest.java | 31 +- .../impl/BrokerClientIntegrationTest.java | 59 ++- .../impl/MessagePublishThrottlingTest.java | 89 +--- .../pulsar/client/impl/PulsarTestClient.java | 4 +- .../impl/TopicPublishThrottlingInitTest.java | 9 - .../pulsar/compaction/CompactionTest.java | 5 +- .../worker/PulsarFunctionTlsTest.java | 16 + .../resources/prometheus_metrics_sample.txt | 2 - .../apache/pulsar/client/api/Consumer.java | 25 + .../client/api/PulsarClientException.java | 4 + .../client/impl/crypto/MessageCryptoBc.java | 2 +- .../apache/pulsar/admin/cli/CmdClusters.java | 1 + .../apache/pulsar/admin/cli/CmdTopics.java | 62 ++- .../pulsar/admin/cli/TestCmdClusters.java | 7 + .../pulsar/admin/cli/TestCmdTopics.java | 102 +++- .../client/impl/BinaryProtoLookupService.java | 18 +- .../apache/pulsar/client/impl/ClientCnx.java | 84 ++-- .../pulsar/client/impl/ConnectionHandler.java | 27 +- .../pulsar/client/impl/ConsumerBase.java | 14 +- .../pulsar/client/impl/ConsumerImpl.java | 15 +- .../pulsar/client/impl/HttpLookupService.java | 5 +- .../pulsar/client/impl/LookupService.java | 6 +- .../pulsar/client/impl/LookupTopicResult.java | 35 ++ .../client/impl/MultiTopicsConsumerImpl.java | 4 +- .../pulsar/client/impl/ProducerImpl.java | 5 +- .../pulsar/client/impl/PulsarClientImpl.java | 27 +- .../impl/BinaryProtoLookupServiceTest.java | 18 +- .../pulsar/client/impl/ClientCnxTest.java | 21 +- .../client/impl/ClientTestFixtures.java | 3 +- .../client/impl/PulsarClientImplTest.java | 5 +- .../client/impl/TopicListWatcherTest.java | 7 +- .../pulsar/common/protocol/Commands.java | 20 +- .../OptionalProxyProtocolDecoder.java | 37 +- .../pulsar/common/util/RateLimiter.java | 286 ----------- pulsar-common/src/main/proto/PulsarApi.proto | 3 + .../pulsar/common/util/RateLimiterTest.java | 248 --------- .../instance/JavaInstanceRunnable.java | 2 +- .../state/BKStateStoreProviderImpl.java | 26 +- .../PulsarMetadataStateStoreProviderImpl.java | 10 +- .../instance/state/StateStoreProvider.java | 19 + .../instance/stats/FunctionStatsManager.java | 17 +- .../instance/stats/SinkStatsManager.java | 17 +- .../instance/stats/SourceStatsManager.java | 17 +- .../functions/worker/LeaderService.java | 4 +- .../functions/worker/PulsarWorkerService.java | 31 +- .../worker/rest/api/ComponentImpl.java | 153 ++---- pulsar-io/alluxio/pom.xml | 51 +- .../pulsar/io/alluxio/sink/AlluxioSink.java | 7 +- .../io/alluxio/sink/AlluxioSinkTest.java | 6 +- pulsar-io/canal/pom.xml | 5 + .../pulsar/io/canal/CanalAbstractSource.java | 2 +- .../pulsar/io/canal/CanalSourceConfig.java | 7 +- .../pulsar/io/common/IOConfigUtils.java | 7 +- .../pulsar/io/common/IOConfigUtilsTest.java | 11 + pulsar-io/dynamodb/pom.xml | 6 + .../pulsar/io/dynamodb/DynamoDBSource.java | 2 +- .../io/dynamodb/DynamoDBSourceConfig.java | 8 +- .../dynamodb/DynamoDBSourceConfigTests.java | 52 +- .../ElasticSearchExtractTests.java | 13 + pulsar-io/hdfs2/pom.xml | 20 +- pulsar-io/hdfs3/pom.xml | 4 + pulsar-io/influxdb/pom.xml | 5 + .../influxdb/InfluxDBGenericRecordSink.java | 4 +- .../io/influxdb/v1/InfluxDBAbstractSink.java | 2 +- .../io/influxdb/v1/InfluxDBSinkConfig.java | 11 +- .../pulsar/io/influxdb/v2/InfluxDBSink.java | 2 +- .../io/influxdb/v2/InfluxDBSinkConfig.java | 14 +- .../influxdb/v1/InfluxDBSinkConfigTest.java | 56 ++- .../influxdb/v2/InfluxDBSinkConfigTest.java | 29 +- pulsar-io/jdbc/core/pom.xml | 6 + .../pulsar/io/jdbc/JdbcAbstractSink.java | 2 +- .../apache/pulsar/io/jdbc/JdbcSinkConfig.java | 7 +- pulsar-io/kafka/pom.xml | 5 + .../pulsar/io/kafka/KafkaAbstractSink.java | 6 +- .../pulsar/io/kafka/KafkaAbstractSource.java | 2 +- .../pulsar/io/kafka/KafkaSinkConfig.java | 11 +- .../pulsar/io/kafka/KafkaSourceConfig.java | 9 +- .../io/kafka/sink/KafkaAbstractSinkTest.java | 8 +- .../kafka/source/KafkaAbstractSourceTest.java | 26 +- pulsar-io/mongo/pom.xml | 5 + .../io/mongodb/MongoAbstractConfig.java | 3 +- .../apache/pulsar/io/mongodb/MongoSink.java | 2 +- .../pulsar/io/mongodb/MongoSinkConfig.java | 9 +- .../apache/pulsar/io/mongodb/MongoSource.java | 2 +- .../pulsar/io/mongodb/MongoSourceConfig.java | 10 +- .../io/mongodb/MongoSinkConfigTest.java | 41 +- .../io/mongodb/MongoSourceConfigTest.java | 38 +- pulsar-io/rabbitmq/pom.xml | 5 + .../pulsar/io/rabbitmq/RabbitMQSink.java | 2 +- .../io/rabbitmq/RabbitMQSinkConfig.java | 9 +- .../pulsar/io/rabbitmq/RabbitMQSource.java | 2 +- .../io/rabbitmq/RabbitMQSourceConfig.java | 7 +- .../rabbitmq/sink/RabbitMQSinkConfigTest.java | 52 +- .../source/RabbitMQSourceConfigTest.java | 60 ++- .../rabbitmq/source/RabbitMQSourceTest.java | 3 +- pulsar-io/redis/pom.xml | 5 + .../pulsar/io/redis/RedisAbstractConfig.java | 5 +- .../pulsar/io/redis/sink/RedisSink.java | 2 +- .../pulsar/io/redis/sink/RedisSinkConfig.java | 11 +- .../io/redis/sink/RedisSinkConfigTest.java | 47 +- .../pulsar/io/redis/sink/RedisSinkTest.java | 5 +- pulsar-io/solr/pom.xml | 5 + .../pulsar/io/solr/SolrAbstractSink.java | 2 +- .../apache/pulsar/io/solr/SolrSinkConfig.java | 7 +- .../pulsar/io/solr/SolrSinkConfigTest.java | 47 +- .../metadata/impl/EtcdMetadataStore.java | 6 +- .../proxy/server/ProxyConfiguration.java | 2 +- .../ProxyWithExtensibleLoadManagerTest.java | 229 +++++++++ pulsar-sql/presto-distribution/LICENSE | 46 +- .../testclient/LoadSimulationController.java | 14 +- .../testclient/PerformanceConsumer.java | 8 + .../testclient/PerformanceProducer.java | 10 +- .../docker-images/java-test-image/Dockerfile | 3 +- .../latest-version-image/Dockerfile | 4 - .../functions/PulsarStateTest.java | 46 +- .../ExtensibleLoadManagerTest.java | 45 +- 254 files changed, 6708 insertions(+), 3548 deletions(-) create mode 100644 microbench/README.md create mode 100644 microbench/pom.xml create mode 100644 microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java rename pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimitFunction.java => microbench/src/main/java/org/apache/pulsar/broker/qos/package-info.java (84%) create mode 100644 microbench/src/main/resources/log4j2.xml create mode 100644 pip/pip-313.md create mode 100644 pip/pip-320.md create mode 100644 pip/pip-322.md create mode 100644 pip/pip-323.md create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/MonotonicSnapshotClock.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/package-info.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PrecisePublishLimiter.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterDisable.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisePublishLimiterTest.java rename pulsar-broker/src/test/java/org/apache/pulsar/broker/service/{PrecisTopicPublishRateThrottleTest.java => TopicPublishRateThrottleTest.java} (72%) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupTopicResult.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/RateLimiterTest.java create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 1331da26ced79..dedaf148d6495 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -104,7 +104,7 @@ function test_group_client() { } function test_group_metadata() { - mvn_test -pl pulsar-metadata + mvn_test -pl pulsar-metadata -DtestReuseFork=false } # prints summaries of failed tests to console diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 9957c3891e450..c92d68ccc8022 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 8.37 3.1.2 - 4.1.100.Final + 4.1.104.Final 4.2.3 32.1.2-jre 1.10.12 diff --git a/conf/bookkeeper.conf b/conf/bookkeeper.conf index 9da459d576e11..548ece01b842d 100644 --- a/conf/bookkeeper.conf +++ b/conf/bookkeeper.conf @@ -255,6 +255,9 @@ rereplicationEntryBatchSize=100 # Enable/disable having read operations for a ledger to be sticky to a single bookie. stickyReadSEnabled=true +# Enable/disable reordering read sequence on reading entries. +reorderReadSequenceEnabled=true + # Auto-replication # The grace period, in milliseconds, that the replication worker waits before fencing and # replicating a ledger fragment that's still being written to upon bookie failure. diff --git a/conf/broker.conf b/conf/broker.conf index 6bef17350eee4..82dd5640740c0 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -539,7 +539,7 @@ brokerServiceCompactionThresholdInBytes=0 brokerServiceCompactionPhaseOneLoopTimeInSeconds=30 # Whether retain null-key message during topic compaction -topicCompactionRemainNullKey=false +topicCompactionRetainNullKey=false # Whether to enable the delayed delivery for messages. # If disabled, messages will be immediately delivered and there will @@ -1015,7 +1015,7 @@ bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false # Enable/disable reordering read sequence on reading entries. -bookkeeperClientReorderReadSequenceEnabled=false +bookkeeperClientReorderReadSequenceEnabled=true # Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie # outside the specified groups will not be used by the broker diff --git a/conf/proxy.conf b/conf/proxy.conf index c41c54670eea4..4194bf7621985 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -370,7 +370,7 @@ zooKeeperCacheExpirySeconds=-1 ### --- Metrics --- ### -# Whether to enable the proxy's /metrics, /proxy-stats, and /status.html http endpoints +# Whether to enable the proxy's /metrics and /proxy-stats http endpoints enableProxyStatsEndpoints=true # Whether the '/metrics' endpoint requires authentication. Defaults to true authenticateMetricsEndpoint=true diff --git a/conf/standalone.conf b/conf/standalone.conf index 30de267b5c76e..cf13f12c8fe6f 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -1288,4 +1288,4 @@ brokerInterceptors= disableBrokerInterceptors=true # Whether retain null-key message during topic compaction -topicCompactionRemainNullKey=false +topicCompactionRetainNullKey=false diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c59090b2882e4..db66fc4ced547 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -289,26 +289,26 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.100.Final.jar - - io.netty-netty-codec-4.1.100.Final.jar - - io.netty-netty-codec-dns-4.1.100.Final.jar - - io.netty-netty-codec-http-4.1.100.Final.jar - - io.netty-netty-codec-http2-4.1.100.Final.jar - - io.netty-netty-codec-socks-4.1.100.Final.jar - - io.netty-netty-codec-haproxy-4.1.100.Final.jar - - io.netty-netty-common-4.1.100.Final.jar - - io.netty-netty-handler-4.1.100.Final.jar - - io.netty-netty-handler-proxy-4.1.100.Final.jar - - io.netty-netty-resolver-4.1.100.Final.jar - - io.netty-netty-resolver-dns-4.1.100.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.100.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.100.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.100.Final.jar - - io.netty-netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.100.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar + - io.netty-netty-buffer-4.1.104.Final.jar + - io.netty-netty-codec-4.1.104.Final.jar + - io.netty-netty-codec-dns-4.1.104.Final.jar + - io.netty-netty-codec-http-4.1.104.Final.jar + - io.netty-netty-codec-http2-4.1.104.Final.jar + - io.netty-netty-codec-socks-4.1.104.Final.jar + - io.netty-netty-codec-haproxy-4.1.104.Final.jar + - io.netty-netty-common-4.1.104.Final.jar + - io.netty-netty-handler-4.1.104.Final.jar + - io.netty-netty-handler-proxy-4.1.104.Final.jar + - io.netty-netty-resolver-4.1.104.Final.jar + - io.netty-netty-resolver-dns-4.1.104.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.104.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.104.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.104.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.104.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.104.Final.jar + - io.netty-netty-transport-native-epoll-4.1.104.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.104.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.104.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -316,9 +316,9 @@ The Apache Software License, Version 2.0 - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - io.netty-netty-tcnative-classes-2.0.61.Final.jar - - io.netty.incubator-netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar + - io.netty.incubator-netty-incubator-transport-classes-io_uring-0.0.24.Final.jar + - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar + - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar * Prometheus client - io.prometheus.jmx-collector-0.16.1.jar - io.prometheus-simpleclient-0.16.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index eeaa33589d640..bdd1b18ce0c3d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -344,22 +344,22 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.21.jar * Netty - - netty-buffer-4.1.100.Final.jar - - netty-codec-4.1.100.Final.jar - - netty-codec-dns-4.1.100.Final.jar - - netty-codec-http-4.1.100.Final.jar - - netty-codec-socks-4.1.100.Final.jar - - netty-codec-haproxy-4.1.100.Final.jar - - netty-common-4.1.100.Final.jar - - netty-handler-4.1.100.Final.jar - - netty-handler-proxy-4.1.100.Final.jar - - netty-resolver-4.1.100.Final.jar - - netty-resolver-dns-4.1.100.Final.jar - - netty-transport-4.1.100.Final.jar - - netty-transport-classes-epoll-4.1.100.Final.jar - - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.100.Final.jar - - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar + - netty-buffer-4.1.104.Final.jar + - netty-codec-4.1.104.Final.jar + - netty-codec-dns-4.1.104.Final.jar + - netty-codec-http-4.1.104.Final.jar + - netty-codec-socks-4.1.104.Final.jar + - netty-codec-haproxy-4.1.104.Final.jar + - netty-common-4.1.104.Final.jar + - netty-handler-4.1.104.Final.jar + - netty-handler-proxy-4.1.104.Final.jar + - netty-resolver-4.1.104.Final.jar + - netty-resolver-dns-4.1.104.Final.jar + - netty-transport-4.1.104.Final.jar + - netty-transport-classes-epoll-4.1.104.Final.jar + - netty-transport-native-epoll-4.1.104.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.104.Final.jar + - netty-transport-native-unix-common-4.1.104.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -367,12 +367,12 @@ The Apache Software License, Version 2.0 - netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - netty-tcnative-classes-2.0.61.Final.jar - - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar - - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.100.Final.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar + - netty-incubator-transport-classes-io_uring-0.0.24.Final.jar + - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar + - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.104.Final.jar + - netty-resolver-dns-native-macos-4.1.104.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.104.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 2bd6d402f7694..c4832f11cfecb 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -54,12 +54,13 @@ FROM ubuntu:22.04 ARG DEBIAN_FRONTEND=noninteractive ARG UBUNTU_MIRROR=http://archive.ubuntu.com/ubuntu/ ARG UBUNTU_SECURITY_MIRROR=http://security.ubuntu.com/ubuntu/ +ARG DEFAULT_USERNAME=pulsar ARG JDK_MAJOR_VERSION=17 # Install some utilities RUN sed -i -e "s|http://archive\.ubuntu\.com/ubuntu/|${UBUNTU_MIRROR:-http://archive.ubuntu.com/ubuntu/}|g" \ -e "s|http://security\.ubuntu\.com/ubuntu/|${UBUNTU_SECURITY_MIRROR:-http://security.ubuntu.com/ubuntu/}|g" /etc/apt/sources.list \ - && echo 'Acquire::http::Timeout "30";\nAcquire::ftp::Timeout "30";\nAcquire::Retries "3";' > /etc/apt/apt.conf.d/99timeout_and_retries \ + && echo 'Acquire::http::Timeout "30";\nAcquire::http::ConnectionAttemptDelayMsec "2000";\nAcquire::https::Timeout "30";\nAcquire::https::ConnectionAttemptDelayMsec "2000";\nAcquire::ftp::Timeout "30";\nAcquire::ftp::ConnectionAttemptDelayMsec "2000";\nAcquire::Retries "15";' > /etc/apt/apt.conf.d/99timeout_and_retries \ && apt-get update \ && apt-get -y dist-upgrade \ && apt-get -y install netcat dnsutils less procps iputils-ping \ @@ -106,4 +107,5 @@ RUN chmod +x /pulsar/bin/install-pulsar-client.sh RUN /pulsar/bin/install-pulsar-client.sh # The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value. +RUN useradd ${DEFAULT_USERNAME} -u 10000 -g 0 USER 10000 diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 1e5d67871d434..8ce2a6924ebed 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -3209,7 +3209,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct } } - private void offloadLoop(CompletableFuture promise, Queue ledgersToOffload, + void offloadLoop(CompletableFuture promise, Queue ledgersToOffload, PositionImpl firstUnoffloaded, Optional firstError) { State currentState = getState(); if (currentState == State.Closed) { @@ -3257,6 +3257,7 @@ private void offloadLoop(CompletableFuture promise, Queue key.equals(FaultInjectionMetadataStore.OperationType.PUT) && + metadataPutCallCount.incrementAndGet() == 2); + + // prepare the arguments for the offloadLoop method + CompletableFuture future = new CompletableFuture<>(); + Queue ledgersToOffload = new LinkedList<>(); + LedgerInfo ledgerInfo = LedgerInfo.getDefaultInstance().toBuilder().setLedgerId(1).setEntries(10).build(); + ledgersToOffload.add(ledgerInfo); + PositionImpl firstUnoffloaded = new PositionImpl(1, 0); + Optional firstError = Optional.empty(); + + // mock the read handle to make the offload successful + CompletableFuture readHandle = new CompletableFuture<>(); + readHandle.complete(mock(ReadHandle.class)); + when(ml.getLedgerHandle(eq(ledgerInfo.getLedgerId()))).thenReturn(readHandle); + when(ledgerOffloader.offload(any(), any(), anyMap())).thenReturn(CompletableFuture.completedFuture(null)); + + ml.ledgers.put(ledgerInfo.getLedgerId(), ledgerInfo); + + // do the offload + ml.offloadLoop(future, ledgersToOffload, firstUnoffloaded, firstError); + + // waiting for the offload complete + try { + future.join(); + fail("The offload should fail"); + } catch (Exception e) { + // the offload should fail + assertTrue(e.getCause().getMessage().contains("mock completion error")); + } + + // the ledger deletion shouldn't happen + verify(ledgerOffloader, times(0)) + .deleteOffloaded(eq(ledgerInfo.getLedgerId()), any(), anyMap()); + } } diff --git a/microbench/README.md b/microbench/README.md new file mode 100644 index 0000000000000..780e3a5a1d3e8 --- /dev/null +++ b/microbench/README.md @@ -0,0 +1,43 @@ + + +# Microbenchmarks for Apache Pulsar + +This module contains microbenchmarks for Apache Pulsar. + +## Running the benchmarks + +The benchmarks are written using [JMH](http://openjdk.java.net/projects/code-tools/jmh/). To compile & run the benchmarks, use the following command: + +```bash +# Compile everything for creating the shaded microbenchmarks.jar file +mvn -Pcore-modules,microbench,-main -T 1C clean package + +# run the benchmarks using the standalone shaded jar in any environment +java -jar microbench/target/microbenchmarks.jar +``` + +For fast recompiling of the benchmarks (without compiling Pulsar modules) and creating the shaded jar, you can use the following command: + +```bash +mvn -Pmicrobench -pl microbench clean package +``` + diff --git a/microbench/pom.xml b/microbench/pom.xml new file mode 100644 index 0000000000000..a62876e8802f7 --- /dev/null +++ b/microbench/pom.xml @@ -0,0 +1,133 @@ + + + 4.0.0 + + + org.apache.pulsar + pulsar + 3.2.0-SNAPSHOT + ../pom.xml + + + microbench + jar + Pulsar Microbenchmarks + + + 1.37 + + + + + microbench + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + microbenchmarks + + + + org.openjdk.jmh.Main + true + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + + + + + org.openjdk.jmh + jmh-core + ${jmh.version} + + + org.openjdk.jmh + jmh-generator-annprocess + ${jmh.version} + provided + + + ${project.groupId} + pulsar-broker + ${project.version} + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-surefire-plugin + + true + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + org.openjdk.jmh + jmh-generator-annprocess + ${jmh.version} + + + + + + + \ No newline at end of file diff --git a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java new file mode 100644 index 0000000000000..4c069e72ea3ba --- /dev/null +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import java.util.concurrent.TimeUnit; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +@Fork(3) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@State(Scope.Thread) +public class AsyncTokenBucketBenchmark { + private AsyncTokenBucket asyncTokenBucket; + private DefaultMonotonicSnapshotClock monotonicSnapshotClock = + new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos(8), System::nanoTime); + + @Setup(Level.Iteration) + public void setup() { + long ratePerSecond = 100_000_000; + asyncTokenBucket = AsyncTokenBucket.builder().rate(ratePerSecond).clock(monotonicSnapshotClock) + .initialTokens(2 * ratePerSecond).capacity(2 * ratePerSecond).build(); + } + + @TearDown(Level.Iteration) + public void teardown() { + monotonicSnapshotClock.close(); + } + + @Threads(1) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void consumeTokensBenchmark001Threads() { + asyncTokenBucket.consumeTokens(1); + } + + @Threads(10) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void consumeTokensBenchmark010Threads() { + asyncTokenBucket.consumeTokens(1); + } + + @Threads(100) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void consumeTokensBenchmark100Threads() { + asyncTokenBucket.consumeTokens(1); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimitFunction.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/package-info.java similarity index 84% rename from pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimitFunction.java rename to microbench/src/main/java/org/apache/pulsar/broker/qos/package-info.java index 0a4b62e9500f9..ccea21a210f86 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimitFunction.java +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/package-info.java @@ -16,11 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.common.util; - /** - * Function use when rate limiter renew permit. - * */ -public interface RateLimitFunction { - void apply(); -} + * Benchmarks for Pulsar broker Quality of Service (QoS) related classes. + */ +package org.apache.pulsar.broker.qos; \ No newline at end of file diff --git a/microbench/src/main/resources/log4j2.xml b/microbench/src/main/resources/log4j2.xml new file mode 100644 index 0000000000000..7ec5ed8169a66 --- /dev/null +++ b/microbench/src/main/resources/log4j2.xml @@ -0,0 +1,33 @@ + + + + + + + + + + + + + + \ No newline at end of file diff --git a/pip/README.md b/pip/README.md index 7b0f4c6c57aa9..f386647e8c5c2 100644 --- a/pip/README.md +++ b/pip/README.md @@ -85,6 +85,31 @@ The process works in the following way: All the future implementation Pull Requests that will be created, should always reference the PIP-XXX in the commit log message and the PR title. It is advised to create a master GitHub issue to formulate the execution plan and track its progress. +### Example +* Eve ran into some issues with the client metrics - she needed a metric which was missing. +* She read the code a bit, and has an idea what metrics she wishes to add. +* She summarized her idea and direction in an email to the DEV mailing list (she located it on +[Discussions]([url](https://pulsar.apache.org/community/#section-discussions)) section on the website. +* She didn't get any response from the community, so she joined the next +[community meeting]([url](https://github.com/apache/pulsar/wiki/Community-Meetings)). There Matteo Merli and Asaf helped +setup a channel in Slack to brainstorm the idea and meet on Zoom with a few Pulsar contributors (e.g. Lari and Tison). +* Once Eve had a good enough context, and good design outline, she opened a new branch in her Pulsar repository, duplicated +TEMPLATE.md and created pip-xxx.MD (the number she will take later). +* She followed the template and submitted the pip as a new PR to pulsar repository. +* Once the PR was created, she modified the version to match the rules described at step 2, both for PR title and file name. +* She sent an email to the DEV mailing list, titled "[DISCUSS] PIP-123: Adding metrics for ..." , described shortly in the +email what the PIP was about and gave a link. +* She got no response for anyone for 2 weeks, so she nudged the people that helped + her brainstorm (e.g. Lary and Tison) and pinged in #dev that she needs more reviewers. +* Once she got 3 reviews from PMC members and the community had at least a few days from the moment + the PR was announceed on DEV, she sent a vote email to the DEV mailing list titled + "[VOTE] PIP-123: Adding metrics for ...". +* She nudged the reviewers to reply with a binding vote, waited for 2-3 days, and then + concluded the vote by sending a reply tallying up the binding and non-binding votes. +* She updated the PIP with links to discuss and vote emails, and then asked a PMC member + who voted +1, to merge (using GitHub mentionon the PR). + + ## List of PIPs ### Historical PIPs diff --git a/pip/pip-300.md b/pip/pip-300.md index 3490ed1bbf915..0be8cf191fb06 100644 --- a/pip/pip-300.md +++ b/pip/pip-300.md @@ -12,7 +12,9 @@ the `pulsar-admin` to change the values of customized configuration, but the Pul ## In Scope -The goal of this PIP is to allow the `pulsar-admin` to update the values of customized configuration. +The goal of this PIP is to allow the `pulsar-admin` to update the values of customized configuration, and also use the +listener to listen the customized configuration changes +by `org.apache.pulsar.broker.service.BrokerService.registerConfigurationListener`. # Detailed Design @@ -31,12 +33,7 @@ public void registerCustomDynamicConfiguration(String key, Predicate val if (dynamicConfigurationMap.containsKey(key)) { throw new IllegalArgumentException(key + " already exists in the dynamicConfigurationMap"); } - try { - ServiceConfiguration.class.getDeclaredField(key); - throw new IllegalArgumentException("Only register custom configuration"); - } catch (Exception ignored) { { - - } + ConfigField configField = ConfigField.newCustomConfigField(null); configField.validator = validator; dynamicConfigurationMap.put(key, configField); diff --git a/pip/pip-313.md b/pip/pip-313.md new file mode 100644 index 0000000000000..e43e965a11771 --- /dev/null +++ b/pip/pip-313.md @@ -0,0 +1,76 @@ +# PIP-313: Support force unsubscribe using consumer api + +# Motivation + +As discussed in Issue: https://github.com/apache/pulsar/issues/21451 + +Apache Pulsar provides a messaging queue using a Shared subscription to process unordered messages in parallel using multiple connected consumers. Shared subscription is also commonly used in data processing pipelines where they need to forcefully unsubscribe from the subscription after processing messages on the topic. One example is Pulsar-Storm adapter where [Pulsar spout](https://github.com/apache/pulsar/blob/branch-2.4/pulsar-storm/src/main/java/org/apache/pulsar/storm/PulsarSpout.java#L126) creates Pulsar consumers on a shared subscription for distributed processing and then unsubscribe on the topic. + +However, PulsarSpout always fails to unsubscribe shared subscriptions and it also doesn't close the pulsar consumers if there is more than one consumer connected to the subscription which causes a leaked subscription and consumer for that application. It also causes a backlog on a topic due to failed unsubscribe and application team has to build external service to just address such failures. + +In this usecases, client application can not successfully unsubscribe on a shared subscription when multiple consumers are connected because Pulsar client library first tries to unsubscribe which will not be successful as multiple consumers are still connected on the subscription and eventually Pulsar client lib fails to unsubscribe and close the consumer on the subscription. Because of that none of the consumers can disconnect or unsubscribe from the subscription. This will make it impossible for applications to unsubscribe on a shared subscription and they need an API to forcefully unsubscribe on a shared subscription using consumer API. +We already have the admin-api to unsubscribe forcefully but adding such support in consumer API will allow applications like Pulsar-storm to unsubscribe successfully and also allow consumers to close gracefully. + +# Goals + +Support unsubscribe API with force option in consumer API along with admin API which can help applications to unsubscribe on various subscriptions such as Failover, Shared, Key-Shared. + +# High Level Design + +Consumer API will have additional unsubscribe api with additional flag to enable forceful unsubscribe on a subscription. Pulsar client library will pass the flag to broker while unsubscribing and broker will use it with existing broker side implementation of ubsubscribing forcefully. + + +## Design & Implementation Details + +### (1) Pulsar client library changes + +Add support of unsubscribe api with force option in Consumer API + +``` +Consumer.java + +void unsubscribe(boolean force) throws PulsarClientException; +CompletableFuture unsubscribeAsync(boolean force); +``` + +Calling unsubscribe with force flag will make broker to fence the subscription and disconnect all the consumers forcefully to eventually unsubscribe and delete the subscription. However, reconnection of the consumer can recreate the subscription so, client application should make sure to call force-unsubscribe from all the consumers to eventually delete subscription or disable auto subscription creation based on application usecases. + +### (2) Protobuf changes + +Pulsar client library will pass an additional force flag (with default value =false) to the broker with wire protocol change + +``` +PulsarApi.proto + +message CommandUnsubscribe { + required uint64 consumer_id = 1; + required uint64 request_id = 2; + optional bool force = 3 [default = false]; +} +``` + +### (3) Broker changes + +Broker already supports force delete subscription using admin-api so, broker already has implementation to unsubscribe forcefully but it doesn’t have option to trigger using binary api. Therefore, once client sends additional force flag to broker while unsubscribing , broker reads the flag and passes to the subscription API to forcefully unsubscribe the subscription. + + +# Security Considerations + + + +# General Notes + +# Links + +Issue: https://github.com/apache/pulsar/issues/21451 +Sample PR: https://github.com/apache/pulsar/compare/master...rdhabalia:shared_unsub?expand=1 +Discuss thread: https://lists.apache.org/thread/hptx8z9mktn94gvqtt4547wzcfcgdsrv +Vote thread: https://lists.apache.org/thread/3kp9hfs5opw17fgmkn251sc6cd408yty + + diff --git a/pip/pip-318.md b/pip/pip-318.md index 2ef558356e0ac..988eea0bb8b36 100644 --- a/pip/pip-318.md +++ b/pip/pip-318.md @@ -25,22 +25,22 @@ If the configuration is true, we will retain null-key messages during topic comp Add config to broker.conf/standalone.conf ```properties -topicCompactionRemainNullKey=false +topicCompactionRetainNullKey=false ``` # Backward & Forward Compatibility -- Make `topicCompactionRemainNullKey=false` default in the 3.2.0. -- Cherry-pick it to a branch less than 3.2.0 make `topicCompactionRemainNullKey=true` default. -- Delete the configuration `topicCompactionRemainNullKey` in 3.3.0 and don't supply an option to retain null-keys. +- Make `topicCompactionRetainNullKey=false` default in the 3.2.0. +- Cherry-pick it to a branch less than 3.2.0 make `topicCompactionRetainNullKey=true` default. +- Delete the configuration `topicCompactionRetainNullKey` in 3.3.0 and don't supply an option to retain null-keys. ## Revert -Make `topicCompactionRemainNullKey=true` in broker.conf/standalone.conf. +Make `topicCompactionRetainNullKey=true` in broker.conf/standalone.conf. ## Upgrade -Make `topicCompactionRemainNullKey=false` in broker.conf/standalone.conf. +Make `topicCompactionRetainNullKey=false` in broker.conf/standalone.conf. # Links diff --git a/pip/pip-320.md b/pip/pip-320.md new file mode 100644 index 0000000000000..3c169e4340bd1 --- /dev/null +++ b/pip/pip-320.md @@ -0,0 +1,256 @@ +# PIP-320 OpenTelemetry Scaffolding + +# Background knowledge + +## PIP-264 - parent PIP titled "Enhanced OTel-based metric system" +[PIP-264](https://github.com/apache/pulsar/pull/21080), which can also be viewed [here](pip-264.md), describes in high +level a plan to greatly enhance Pulsar metric system by replacing it with [OpenTelemetry](https://opentelemetry.io/). +You can read in the PIP the numerous existing problems PIP-264 solves. Among them are: +- Control which metrics to export per topic/group/namespace via the introduction of a metric filter configuration. + This configuration is planned to be dynamic as outline in the [PIP-264](pip-264.md). +- Reduce the immense metrics cardinality due to high topic count (One of Pulsar great features), by introducing +the concept of Metric Group - a group of topics for metric purposes. Metric reporting will also be done to a +group granularity. 100k topics can be downsized to 1k groups. The dynamic metric filter configuration would allow +the user to control which metric group to un-filter. +- Proper histogram exporting +- Clean-up codebase clutter, by relying on a single industry standard API, SDK and metrics protocol (OTLP) instead of +existing mix of home-brew libraries and hard coded Prometheus exporter. +- any many more + +You can [here](pip-264.md#why-opentelemetry) why OpenTelemetry was chosen. + +## OpenTelemetry +Since OpenTelemetry (a.k.a. OTel) is an emerging industry standard, there are plenty of good articles, videos and +documentation about it. In this very short paragraph I'll describe what you need to know about OTel from this PIP +perspective. + +OpenTelemetry is a project aimed to standardize the way we instrument, collect and ship metrics from applications +to telemetry backends, be it databases (e.g. Prometheus, Cortex, Thanos) or vendors (e.g. Datadog, Logz.io). +It is divided into API, SDK and Collector: +- API: interfaces to use to instrument: define a counter, record values to a histogram, etc. +- SDK: a library, available in many languages, implementing the API, and other important features such as +reading the metrics and exporting it out to a telemetry backend or OTel Collector. +- Collector: a lightweight process (application) which can receive or retrieve telemetry, transform it (e.g. +filter, drop, aggregate) and export it (e.g. send it to various backends). The SDK supports out-of-the-box +exporting metrics as Prometheus HTTP endpoint or sending them out using OTLP protocol. Many times companies choose to +ship to the Collector and there ship to their preferred vendors, since each vendor already published their exporter +plugin to OTel Collector. This makes the SDK exporters very light-weight as they don't need to support any +vendor. It's also easier for the DevOps team as they can make OTel Collector their responsibility, and have +application developers only focus on shipping metrics to that collector. + +Just to have some context: Pulsar codebase will use the OTel API to create counters / histograms and records values to +them. So will the Pulsar plugins and Pulsar Function authors. Pulsar itself will be the one creating the SDK +and using that to hand over an implementation of the API where ever needed in Pulsar. Collector is up to the choice +of the user, as OTel provides a way to expose the metrics as `/metrics` endpoint on a configured port, so Prometheus +compatible scrapers can grab it from it directly. They can also send it via OTLP to OTel collector. + +## Telemetry layers +PIP-264 clearly outlined there will be two layers of metrics, collected and exported, side by side: OpenTelemetry +and the existing metric system - currently exporting in Prometheus. This PIP will explain in detail how it will work. +The basic premise is that you will be able to enable or disable OTel metrics, alongside the existing Prometheus +metric exporting. + +## Why OTel in Pulsar will be marked experimental and not GA +As specified in [PIP-264](pip-264.md), OpenTelemetry Java SDK has several fixes the Pulsar community must +complete before it can be used in production. They are [documented](pip-264.md#what-we-need-to-fix-in-opentelemetry) +in PIP-264. The most important one is reducing memory allocations to be negligible. OTel SDK is built upon immutability, +hence allocated memory in O(`#topics`) which is a performance killer for low latency application like Pulsar. + +You can track the proposal and progress the Pulsar and OTel communities are making in +[this issue](https://github.com/open-telemetry/opentelemetry-java/issues/5105). + + +## Metrics endpoint authentication +Today Pulsar metrics endpoint `/metrics` has an option to be protected by the configured `AuthenticationProvider`. +The configuration option is named `authenticateMetricsEndpoint` in the broker and +`authenticateMetricsEndpoint` in the proxy. + + +# Motivation + +Implementing PIP-264 consists of a long list of steps, which are detailed in +[this issue](https://github.com/apache/pulsar/issues/21121). The first step is add all the bare-bones infrastructure +to use OpenTelemetry in Pulsar, such that next PRs can use it to start translating existing metrics to their +OTel form. It means the same metrics will co-exist in the codebase and also in runtime, if OTel was enabled. + +# Goals + +## In Scope +- Ability to add metrics using OpenTelemetry to Pulsar components: Broker, Function Worker and Proxy. +- User can disable or enable OpenTelemetry metrics, which by default will be disabled +- OpenTelemetry metrics will be configured via its native OTel Java SDK configuration options +- All the necessary information to use OTel with Pulsar will be documented in Pulsar documentation site +- OpenTelemetry metrics layer defined as experimental, and *not* GA + + +## Out of Scope +- Ability to add metrics using OpenTelemetry as Pulsar Function author. +- Only authenticated sessions can access OTel Prometheus endpoint, using Pulsar authentication +- Metrics in Pulsar clients (as defined in [PIP-264](pip-264.md#out-of-scope))) + +# High Level Design + +## Configuration +OpenTelemetry, as any good telemetry library (e.g. log4j, logback), has its own configuration mechanisms: +- System properties +- Environment variables +- Experimental file-based configuration + +Pulsar doesn't need to introduce any additional configuration. The user can decide, using OTel configuration +things like: +* How do I want to export the metrics? Prometheus? Which port prometheus will be exposed at +* Change histogram buckets using Views +* and more + +Pulsar will use `AutoConfiguredOpenTelemetrySdk` which uses all the above configuration mechanisms +(documented [here](https://github.com/open-telemetry/opentelemetry-java/tree/main/sdk-extensions/autoconfigure)). +This class builds an `OpenTelemetrySdk` based on configurations. This is the entry point to OpenTelemetry API, as it +implements `OpenTelemetry` API class. + +### Setting sensible defaults for Pulsar +There are some configuration options we wish to change their default, but still allow the users to override it +if they wish. We think those default values will make a much easier user experience. + +* `otel.experimental.metrics.cardinality.limit` - value: 10,000 +This property sets an upper bound on the amount of unique `Attributes` an instrument can have. Take Pulsar for example, +an instrument like `pulsar.broker.messaging.topic.received.size`, the unique `Attributes` would be in the amount of +active topics in the broker. Since Pulsar can handle up to 1M topics, it makes more sense to put the default value +to 10k, which translates to 10k topics. + +`AutoConfiguredOpenTelemetrySdkBuilder` allows to add properties using the method `addPropertiesSupplier`. The +System properties and environment variables override it. The file-based configuration still doesn't take +those properties supplied into account, but it will. + + +## Opting in +We would like to have the ability to toggle OpenTelemetry-based metrics, as they are still new. +We won't need any special Pulsar configuration, as OpenTelemetry SDK comes with a configuration key to do that. +Since OTel is still experimental, it will have to be opt-in, hence we will add the following property to be the default +using the mechanism described [above](#setting-sensible-defaults-for-pulsar): + +* `otel.sdk.disabled` - value: true + This property value disables OpenTelemetry. + +With OTel disabled, the user remains with the existing metrics system. OTel in a disabled state operates in a +no-op mode. This means, instruments do get built, but the instrument builders return the same instance of a +no-op instrument, which does nothing on record-values method (e.g. `add(number)`, `record(number)`). The no-op +`MeterProvider` has no registered `MetricReader` hence no metric collection will be made. The memory impact +is almost 0 and the same goes for CPU impact. + +The current metric system doesn't have a toggle which causes all existing data structures to stop collecting +data. Inserting will need changing in so many places since we don't have a single place which through +all metric instrument are created (one of the motivations for PIP-264). +The current system do have a toggle: `exposeTopicLevelMetricsInPrometheus`. It enables toggling off +topic-level metrics, which means the highest cardinality metrics will be namespace level. +Once that toggle is `false`, the amount of data structures accounting memory would in the range of +a few thousands which shouldn't post a burden memory wise. If the user refrain from calling +`/metrics` it will also reduce the CPU and memory cost associated with collecting metrics. + +When the user enables OTel it means there will be a memory increase, but if the user disabled topic-level +metrics in existing system, as specified above, the majority of the memory increase will be due to topic level +metrics in OTel, at the expense of not having them in the existing metric system. + + + +## Cluster attribute name +A broker is part of a cluster. It is configured in the Pulsar configuration key `clusterName`. When the broker is part +of a cluster, it means it shares the topics defined in that cluster (persisted in Metadata service: e.g. ZK) +among the brokers of that cluster. + +Today, each unique time series emitted in Prometheus metrics contains the `cluster` label (almost all of them, as it +is done manually). We wish the same with OTel - to have that attribute in each exported unique time series. + +OTel has the perfect location to place attributes which are shared across all time series: Resource. An application +can have multiple Resource, with each having 1 or more attributes. You define it once, in OTel initialization or +configuration. It can contain attributes like the hostname, AWS region, etc. The default contains the service name +and some info on the SDK version. + +Attributes can be added dynamically, through `addResourceCustomizer()` in `AutoConfiguredOpenTelemetrySdkBuilder`. +We will use that to inject the `cluster` attribute, taken from the configuration. + +In Prometheus, we submitted a [proposal](https://github.com/open-telemetry/opentelemetry-specification/pull/3761) +to opentelemetry specifications, which was merged, to allow copying resource attributes into each exported +unique time series in Prometheus exporter. +We plan to contribute its implementation to OTel Java SDK. + +Resources in Prometheus exporter, are exported as `target_info{} 1` and the attributes are added to this +time series. This will require making joins to get it, making it extremely difficult to use. +The other alternative was to introduce our own `PulsarAttributesBuilder` class, on top of +`AttributesBuilder` of OTel. Getting every contributor to know this class, use it, is hard. Getting this +across Pulsar Functions or Plugins authors, will be immensely hard. Also, when exporting as +OTLP, it is very inefficient to repeat the attribute across all unique time series, instead of once using +Resource. Hence, this needed to be solved in the Prometheus exporter as we did in the proposal. + +The attribute will be named `pulsar.cluster`, as both the proxy and the broker are part of this cluster. + +## Naming and using OpenTelemetry + +### Attributes +* We shall prefix each attribute with `pulsar.`. Example: `pulsar.topic`, `pulsar.cluster`. + +### Instruments +We should have a clear hierarchy, hence use the following prefix +* `pulsar.broker` +* `pulsar.proxy` +* `pulsar.function_worker` + +### Meter +It's customary to use reverse domain name for meter names. Hence, we'll use: +* `org.apache.pulsar.broker` +* `org.apache.pulsar.proxy` +* `org.apache.pulsar.function_worker` + +OTel meter name is converted to the attribute name `otel_scope_name` and added to each unique time series +attributes by Prometheus exporter. + +We won't specify a meter version, as it is used solely to signify the version of the instrumentation, and +currently we are the first version, hence not use it. + + +# Detailed Design + +## Design & Implementation Details + +* `OpenTelemetryService` class + * Parameters: + * Cluster name + * What it will do: + - Override default max cardinality to 10k + - Register a resource with cluster name + - Place defaults setting to instruct Prometheus Exporter to copy resource attributes + - In the future: place defaults for Memory Mode to be REUSABLE_DATA + +* `PulsarBrokerOpenTelemetry` class + * Initialization + * Construct an `OpenTelemetryService` using the cluster name taken from the broker configuration + * Constructs a Meter for the broker metrics + * Methods + * `getMeter()` returns the `Meter` for the broker + * Notes + * This is the class that will be passed along to other Pulsar service classes that needs to define + telemetry such as metrics (in the future: traces). + +* `PulsarProxyOpenTelemetry` class + * Same as `PulsarBrokerOpenTelemetry` but for Pulsar Proxy +* `PulsarWorkerOpenTelemetry` class + * Same as `PulsarBrokerOpenTelemetry` but for Pulsar function worker + + +## Public-facing Changes + +### Public API +* OTel Prometheus Exporter adds `/metrics` endpoint on a user defined port, if user chose to use it + +### Configuration +* OTel configurations are used + +# Security Considerations +* OTel currently does not support setting a custom Authenticator for Prometheus exporter. +An issue has been raised [here](https://github.com/open-telemetry/opentelemetry-java/issues/6013). + * Once it do we can secure the Prometheus exporter metrics endpoint using `AuthenticationProvider` +* Any user can access metrics, and they are not protected per tenant. Like today's implementation + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/xcn9rm551tyf4vxrpb0th0wj0kktnrr2 +* Mailing List voting thread: https://lists.apache.org/thread/zp6vl9z9dhwbvwbplm60no13t8fvlqs2 diff --git a/pip/pip-322.md b/pip/pip-322.md new file mode 100644 index 0000000000000..6a18567ea9012 --- /dev/null +++ b/pip/pip-322.md @@ -0,0 +1,406 @@ +# PIP-322: Pulsar Rate Limiting Refactoring + +# Motivation + +The current rate limiting implementation in Apache Pulsar has several +known issues that impact performance and accuracy when operating Pulsar +clusters under load (detailed in [Problems to +Address](#problems-to-address)). This proposal outlines a refactor to +consolidate multiple existing options into a single improved solution. + +The refactor aims to resolve numerous user complaints regarding default +Pulsar rate limiting being unusable. In addition, inconsistencies and +thread contention with existing rate limiters cause unpredictable +throttling and added latency. + +Refactoring the built-in implementation will improve multi-tenancy, +allow Pulsar to scale to demanding workloads, and address longstanding +issues. + +Rate limiters act as a conduit to more extensive capacity management and +Quality of Service (QoS) controls in Pulsar. They are integral to +Pulsar's core multi-tenancy features. This refactoring will pave the way +for future enhancements. + +# Goals + +## In Scope + +- Preserve current functionality without breaking changes +- Consolidate the multiple existing rate limiting options into a single, + configurable rate limiting solution +- Remove the separate “precise” rate limiter + +### Problems to Address + +- High CPU load with default rate limiter +- High lock contention that impacts shared Netty IO threads and adds + latency to unrelated Pulsar topic producers ([[Bug] RateLimiter lock + contention when use precise publish rate limiter + #21442](https://github.com/apache/pulsar/issues/21442).) +- Multiple limiting implementations (default, precise) which + unnecessarily expose implementation details to users of Pulsar and + make the code harder to maintain and improve +- Inability to limit throughput consistently when using default rate + limiter +- Inconsistent behavior across multiple levels of throttling (broker, + namespace, connection) +- Code maintainability + - Improve understandability of code + +## Out of Scope + +- Custom/pluggable rate limiters +- Additional rate limiting features +- Cluster-wide capacity management +- Addressing the shared connection multiplexing problem where throttling + multiple independent streams multiplexed on the same connection cannot + be consistently throttled by pausing reads on the server side. + +# Current solution + +## Rate limiters in Pulsar + +In Pulsar, rate limiters are used for a few cases: + - publisher rate limiting + - topic level (configured at namespace level or topic level policy + with admin api) + - broker level (configured in broker.conf) + - resource group level (configured with resource groups admin api) + - dispatcher rate limiting + - subscribe rate limiting + - namespace bundle unloading rate limiting + +For producers ("publishers"), there are addition conditions to throttle, +besides the rate limiters: + - limiting pending publish requests per connection, configured with + `maxPendingPublishRequestsPerConnection` in broker configuration + - limiting memory for publishing messages, configured with + `maxMessagePublishBufferSizeInMB` in broker configuration + +### Current publisher rate limiters in Pulsar + +Pulsar contains two implementations for publisher rate limiters: +"default" and "precise". "precise" is the rate limiter implementation +which is used when the broker is configured with +`preciseTopicPublishRateLimiterEnable=true` in broker.conf. + +#### Default publisher rate limiter + +In this approach, a sub-second scheduler runs (configured with +`brokerPublisherThrottlingTickTimeMillis`, defaults to 50ms), iterating +every topic in the broker and checking if the topic has exceeded its +threshold. If so, it will toggle the autoread state of the connection +for the client's producer. Concurrently, a separate one-second scheduler +resets the counters and re-enables throttled connections. This method +results in inaccurate rate limiting. Additionally, this approach can +result in increased CPU usage due to the operation of two schedulers +which are constantly iterating all topics and toggling autoread states. + +#### Precise publisher rate limiter + +In this approach, the rate limit check is done on every send messages +request and thus the rate limiting is enforced more accurately. This +fixes the main issues of the default rate limiters. However, it +introduces a lock contention problem since the rate limiter +implementation extensively uses synchronous methods. Since this lock +content happens on Netty IO threads, it impacts also unrelated topics on +the same broker and causes unnecessary slowdowns as reported by bug +[#21442](https://github.com/apache/pulsar/issues/21442). + +### Publisher Throttling Approach + +In the Pulsar binary protocol, the broker's only method of applying +backpressure to the client is to pause reads and allow the buffers to +fill up. There is no explicit protocol-level, permit-based flow control +as there is for consumers. + +When the broker throttles a producer, it needs to pause reading on the +connection that the client's producer is using. This is achieved by +setting the Netty channel's autoread state to false. + +The broker cannot reject a message that is already in progress. Pausing +reading on the connection prevents the broker from receiving new +messages and this throttles publishing. When reading is paused, Netty +channel and OS-level TCP/IP buffers will fill up and eventually signal +backpressure on the TCP/IP level to the client side. + +In the current solution, when the rate limit is exceeded, the autoread +state is set to false for all producers. Similarly, when the rate falls +below the limit, the autoread state is set to true. When the +broker-level limit is exceeded or falls below the limit, all producers +in the entire broker are iterated. At the topic level, it's for all +producers for the topic. In the resource group, it's all producers part +of the resource group. + +This current solution therefore spends CPU cycles to iterate through the +producers and toggle the autoread flags. It's not necessary to eagerly +iterate all producers in a publisher rate limiter. Rate limiting can +also be achieved when producers are lazily throttled only after they +have sent a publishing send request +([CommandSend](https://pulsar.apache.org/docs/next/developing-binary-protocol/#command-send)) +to the broker. + +It's perfectly acceptable to throttle only active producers one by one +after new messages have arrived when the rate limit has been exceeded. +The end result is the same: the target rate can be kept under the limit. +The calculated rate is always an average rate over a longer period of +time. This is true in both the existing solution and the proposed +solution. Over a very short time span, the observed rate can be +extremely high. This all smoothens out when the rate calculation spans +over hundreds of milliseconds or longer periods. This is why it's +perfectly fine to throttle producers as they produce messages. The +proposed solution accounts all traffic in the rate limiter since the +state is preserved over the rate limiting period (1 second) and isn't +resetted as it is in the current solution which will miss accounting for +traffic around the boundary when the limit has exceeded, but the +connections haven't yet been paused. That's yet another reason why the +lazy approach is suitable for the new proposed solution. + +The externally observable behavior of the rate limiting is actually +better than before since it is possible to achieve fairness in a +producer throttling solution that is implemented in this approach. +Fairness is a general property that is expected in resource sharing +approaches such that each resource consumer is given a similar share of +the resource. In the current publisher rate limiting solution, there's +no way to achieve fairness when the rate limit is being exceeded. In the +proposed solution, fairness is achieved by using a queue to track which +producer is given a turn to produce while the rate limit has been +exceeded and producers are throttled. If the rate limit is again +exceeded, the producer will be put back into the queue and wait for its +turn until it can produce again. In the current solution, the producers +are iterated in the order that they appear in the broker's registry. The +producers at the beginning get more chances to produce than the ones +that are further down the list. The impact of this is the lack of +fairness. + +# High-Level Design + +The proposed refactor will refactor rate limiting internals while +preserving existing user-facing public APIs and user-facing behavior. A +token bucket algorithm will provide efficient and accurate calculations +to throttle throughput. + +Multiple built-in options such as "precise" rate limiter will be +consolidated under a single solution. Performance issues caused by +contention and CPU overhead will be addressed. + +# Detailed Design + +## Proposed Solution + +### Using an asynchronous token bucket algorithm + +Token bucket algorithms are a common industry practice for handling +traffic shaping. It is well understood and it's conceptually simple. A +token bucket is simply a counter which is limited to a maximum value, +the token bucket's capacity. New tokens are added to the bucket with the +configured rate. The usage consumes tokens from the token bucket. When +the token bucket is empty, the usage should be backpressured. In use +cases where the already accepted work cannot be rejected, the token +value needs to also go to negative values. + +Since token bucket algorithm is essentially a counter where new tokens +are added based on the time that has elapsed since the last token update, +it is possible to implement this algorithm in Java in a lockless, +non-blocking way using compare-and-swap (CAS) operations on volatile +fields. There is no need for a scheduler to add new tokens since the +amount of new tokens to add can be calculated from the elapsed time. +This assumption has already been validated in the +https://github.com/lhotari/async-tokenbucket repository. + +There's no current intention to use async-tokenbucket as a separate +library. The AsyncTokenBucket class will be placed directly in the +Pulsar code base. The reason to have async-tokenbucket repository +separately is to have more detailed performance benchmarks there and a +PoC of the high performance. + +The purpose of the proof-of-concept async-tokenbucket was to ensure that +it has an extremely low overhead which makes it feasible to calculate +the amount of tokens in an eventually consistent manner with a +configurable resolution, without a scheduler. The token bucket +operations won't become a bottleneck since on a Dell XPS 2019 i9 laptop +the benchmark showed about 900M token bucket ops/s and on MBP 2023 M3 +Max it was around 2500M token bucket ops/s. + +Internally AsyncTokenBucket uses an eventual consistent approach to +achieve high performance and low overhead. What this means is that the +token balance is updated once in every interval of the configured +resolutionNanos (16 ms default) or when an explicit update of the +balance is requested. + +There is no separate scheduled task to add new tokens to the bucket. New +tokens are calculated based on the elapsed time since the last update +and added to the current tokens balance as part of the token balance +update that happens when tokens are consumed, the throttling period is +calculated or the token balance is queried. + +For example, when tokens are consumed and the balance hasn't been +updated in the current interval, new tokens will be calculated and added +and limited by the token bucket capacity. The consumed tokens and +pending consumed tokens will be flushed and substracted from the balance +during the update. + +If there was already an update for the tokens balance in the current +internal, the consumed tokens are added to the pending consumed tokens +LongAdder counter which will get flushed in the token balance update. + +This makes the tokens balance eventually consistent. The reason for this +design choice is to optimize performance by preventing CAS loop +contention which could cause excessive CPU consumption. + +Key methods in AsyncTokenBucket: +- `consumeTokens()`: Consumes given number of tokens +- `consumeTokensAndCheckIfContainsTokens()`: Consumes given number of + tokens and checks if any tokens remain +- `containsTokens()`: Checks if any tokens remain +- `calculateThrottlingDuration()`: Computes how long throttling should + last until the token bucket contains at least 16 milliseconds worth of + tokens filled with the configured rate. + +The token balance in AsyncTokenBucket is eventually consistent and +differ from the actual token count by up to 16 milliseconds (default +resolutionNanos) worth of consumption. This is not a problem since when +the throttling finally happens, the strongly consistent value is used +for throttling period calculations and no consumed tokens are missed in +the calculations since the token value can go to negative values too. +The average rate will smoothen out to meet the target rate of the rate +limiter with this eventual consistent solution and it doesn't impact the +externally observable behavior. + +For unit tests, eventual consistent behavior can be a challenge. For +that purpose, its possible to switch the AsyncTokenBucket class to a +strongly consistent mode for unit tests by calling static +`switchToConsistentTokensView` and +`resetToDefaultEventualConsistentTokensView` methods on the class. + +One notable improvement of AsyncTokenBucket is that it is completely +non-blocking and lockless. Using AsyncTokenBucket as the basis for +publishing rate limiters in Pulsar will address a severe performance +bottleneck in Pulsar with the "precise" rate limiter. This is reported +as[[Bug] RateLimiter lock contention when use precise publish rate +limiter #21442](https://github.com/apache/pulsar/issues/21442). + +### Unifying rate limiting implementations and improving code maintainability + +In the proposed solution there's no need for separate "default" and +"precise" rate limiting options. A single implementation will be used. +This improves understandability, code quality and maintainability. + +### Fixing the inconsistency of multiple levels of throttling + +In Pulsar throttling can happen for producers in 5 different ways +simultaneously: publisher rate limiting happens at 3 levels: broker, +topic, resource group and in addition there's backpressure for limiting +number of pending publish requests and limiting memory used for +publishing. (detailed in [Rate limiters in +Pulsar](#rate-limiters-in-pulsar)). + +When there are 5 different simultaneous conditions for throttling a +connection, the connection should be throttled as long as any of these +conditions is present. In the current code base, this handling is prone +to errors and overly complex. There are also cases where one rate +limiter sets the autoread to false and another immediately sets it to +true although the connection should remain throttled as long as one of +the conditions exists. + +The fix for this issue is in the proposal by introducing a new concept +ServerCnxThrottleTracker, which will track the "throttle count". When a +throttling condition is present, the throttle count is increased and +when it's no more present, the count is decreased. The autoread should +be switched to false when the counter value goes from 0 to 1 and only +when it goes back from 1 to 0 should it set to true again. The autoread +flag is no more controlled directly from the rate limiters. Rate +limiters are only responsible for their part and it's +ServerCnxThrottleTracker that decides when autoread flag is toggled. + +### Integrating AsyncTokenBucket with the refactored PublishRateLimiterImpl + +In the refactored PublishRateLimiterImpl, there's a AsyncTokenBucket +instance for the message rate limit and for the bytes rate limit. When +the publish operation starts in the broker, it will call the topic's +incrementPublishCount method and pass the reference to the producer that +is starting the operation, in addition to the number of messages and the +total bytes size of the send request (CommandSend message). + +This delegates to a call for all possibly active rate limiters in the topic, +at the broker level, at resource group level and at topic level. + +For each rate limiter, the PublishRateLimiterImpl's handlePublishThrottling +method will be called which also gets the producer reference and the number of message +and total bytes size as input. + +The rate limiter instance could contain both a message limit and a bytes limit. +It will call AsyncTokenBucket's consumeTokensAndCheckIfContainsTokens method +for each instance. If either call returns false, it means that the +producer that produced the message should be throttled. + +Throttling is handled by calling producer's incrementThrottleCount method +which will be delegated producer's connection's ServerCnxThrottleTracker's +incrementThrottleCount method which was described in the previous section. + +The contract of the incrementThrottleCount method is that decrementThrottleCount +method should be called when the throttling is no longer needed from an +individual PublishRateLimiterImpl instance's perspective. + +This is handled by first adding the throttled producer to a queue. +A task will be scheduled to handle unthrottling from the queue after the +throttling duration which is calculated by calling AsyncTokenBucket's +calculateThrottlingDuration method. This task will only be scheduled +unless there's an already scheduled task in progress. + +When the unthrottling task runs, it will process the unthrottling queue +and keep on unthrottling producers while there are available tokens in the +token buckets. If the queue isn't empty, it will repeat the cycle by +scheduling a new task after the throttling duration calculated with +the calculateThrottlingDuration method. This happens until the queue is +empty and will start again if more producers are throttled. + +The producer's connection will get throttled by setting autoread to +false ServerCnxThrottleTracker. The PublishRateLimiterImpl instances +don't have to know whether the connection was already throttled due to +another effective rate limit being over the limit. +ServerCnxThrottleTracker will also handle setting autoread to true once +all rate limiters operating on the same connection have unthrottled the +producer by calling decrementThrottleCount. + +### Preserve Public Contracts + +- Avoid breaking existing configs, APIs or client functionality. +- Handle through internal refactoring. + +## Public-facing Changes + +There are no changes to existing configs, CLI options, monitoring etc. +This PIP is about a large change which includes a major refactoring and +multiple improvements and bug fixes to rate limiting. + +## More Detailed Level Design + +Please refer directly to [the pull request with the proposed +changes](https://github.com/apache/pulsar/pull/21681) for the more +detailed level changes. + +The implementation level detail questions can be handled in the pull +request review. The goal is to document low level details directly in +the Javadoc and comments so that it serves the code maintainers also in +the future. + +# Links + + +* Mailing List discussion thread: + https://lists.apache.org/thread/xzrp2ypggp1oql437tvmkqgfw2b4ft33 +* Mailing List voting thread: + https://lists.apache.org/thread/bbfncm0hdpx42hrj0b2xnzb5oqm1pwyl +* Proposed changes for Pulsar Rate limiting refactoring: + https://github.com/apache/pulsar/pull/21681 + +* [Pulsar Community Meeting minutes + 2023/11/23](https://lists.apache.org/thread/y1sqpyv37fo0k4bm1ox28wggvkb7pbtw) +* [Blog post: Apache Pulsar service level objectives and rate + limiting](https://codingthestreams.com/pulsar/2023/11/22/pulsar-slos-and-rate-limiting.html) +* Proof-of-concept asynchronous token bucket implementation: + https://github.com/lhotari/async-tokenbucket \ No newline at end of file diff --git a/pip/pip-323.md b/pip/pip-323.md new file mode 100644 index 0000000000000..dc607fff3d58c --- /dev/null +++ b/pip/pip-323.md @@ -0,0 +1,171 @@ +# PIP-323: Complete Backlog Quota Telemetry + +# Background knowledge + +## Backlog + +A topic in Pulsar is the place where messages are written to. They are consumed by subscriptions. A topic can have many +subscriptions, and it is those that maintains the state of message acknowledgment, per subscription - which messages +were acknowledged and which were not. + +A subscription backlog is the set of unacknowledged messages in that subscription. +A subscription backlog size is the sum of the size of the unacknowledged messages (in bytes).. + +Since a topic can have many subscriptions, and each has its own backlog, how does one define a backlog for a topic? +A topic backlog is defined as the backlog of the subscription which has the **oldest** unacknowledged message. +Since acknowledged messages can be interleaved with unacknowledged messages, calculating the exact size of that +subscription backlog can be expensive as it requires I/O operations to read the messages from the ledgers. +For that reason, the topic backlog size is actually defined to be the *estimated* backlog size of that subscription. +It does so by summarizing the size of all the ledgers, starting from the current active one (the one being written to), +up to the ledger which contains the oldest unacknowledged message for that subscription (There is actually a faster +way to calculate it, but this was the definition chosen for this estimation in Pulsar). + +A topic backlog age is the age of the oldest unacknowledged message (same subscription as defined for topic backlog size). +If that message was written 30 minutes ago, its age is 30 minutes, and so is the topic backlog age. + +## Backlog Quota + +Pulsar has a feature called [backlog quota](https://pulsar.apache.org/docs/3.1.x/cookbooks-retention-expiry/#backlog-quotas). +It allows a user to define a quota - in effect, a limit - which limits the topic backlog. +There are two types of quotas: + +1. Size based: The limit is for the topic backlog size (as we defined above). +2. Time based: The limit is for the topic backlog age (as we defined above). + +Once a topic backlog exceeds either one of those limits, an action is taken to hold the backlog to that limit: + +* The producer write is placed on hold for a certain amount of time before failing. +* The producer write is failed +* The subscriptions oldest unacknowledged messages will be acknowledged in-order until both the topic backlog size or + age will fall inside the limit (quota). The process is called backlog eviction (happens every interval). + +The quotas can be defined as a default value for any topic, by using the following broker configuration keys: +`backlogQuotaDefaultLimitBytes` and `backlogQuotaDefaultLimitSecond`. + +The quota can also be specified directly for all topics in a given namespace using the namespace policy, +or a specific topic using a topic policy. + +## Monitoring Backlog Quota + +The user today can calculate quota used for size based limit, since there are two metrics exposed today on +a topic level: `pulsar_storage_backlog_quota_limit` and `pulsar_storage_backlog_size`. +You can just divide the two to get a percentage and know how close the topic backlog to its size limit. + +For the time-based limit, the only metric exposed today is the quota itself - `pulsar_storage_backlog_quota_limit_time` + +## Backlog Quota Eviction in the Broker + +The broker has a method called `BrokerService.monitorBacklogQuota()`. It is scheduled to run every x seconds, +as defined by the configuration `backlogQuotaCheckIntervalInSeconds`. +This method loops over all persistent topics, and for each topic is checks whether the topic backlog exceeded +either one of those topics. + +As mentioned before, checking backlog size is a memory-only calculation, since +each topic has the list of ledgers stored in-memory, including the size of each ledger. Same goes for the subscriptions, +they are all stored in memory, and the `ManagedCursor` keeps track of the subscription with the oldest unacknowledged +message, thus retrieveing it is O(1). Checking backlog based on time is costly if configuration key +`preciseTimeBasedBacklogQuotaCheck` was set to true. In that case, it needs to read the oldest message to obtain +its public timestamp, which is expensive in terms of I/O. If it was set to false, it's in-memory access only, since +it uses the age of the ledger instead of the message, and the ledgers metadata is kept in memory. + +For each topic which has exceeded its quota, if the policy chosen is eviction, then the process it performed +synchronously. This process consumes I/O, as it needs read messages (using skip) to know where to stop acknowledging +messages. + + +# Motivation + +Users which have defined backlog quota based on time, have no means today to monitor the backlog quota usage, +time-wise, to know whether the topic backlog is close to its time limit or even passed it. + +If it has passed it, the user has no means to know if it happened, when and how many times. + + +# Goals + +## In Scope +- Allow the user to know the backlog quota usage for time-based quota, per topic +- Allow the user to know how many times backlog eviction happened, and for which backlog quota type + +## Out of Scope + +None + + +# High Level Design + +We'll use the existing backlog monitoring process running in intervals. For each topic, the subscription with +the oldest unacknowledged message is retrieved, to calculate the topic backlog age. At that point, we will +cache the following for the oldest unacknowledged message: +* Subscription name +* Message position +* Message publish timestamp + +That cache will allow us to add a metric exposing the topic backlog age - `pulsar_storage_backlog_age_seconds`, +which will be both consistent (same ones used for deciding on backlog eviction) and cheap to retrieve +(no additional I/O involved). +Coupled with the existing `pulsar_storage_backlog_quota_limit_time` metric, the user can use both to divide and +get the usage of the quota (both are in seconds units). + +We will add the subscription name containing the oldest unacknowledged message to the Admin API +topic stats endpoints (`{tenant}/{namespace}/{topic}/stats` and `{tenant}/{namespace}/{topic}/partitioned-stats`), +allowing the user a complete workflow: alert using metrics when topic backlog is about to be exceeded, then +query topic stats for that topic to retrieve the subscription name which contains the oldest message. +For completeness, we will also add the backlog quota limits, both age and size, and the age of oldest +unacknowledged message. + +We will add a metric allowing the user to know how many times the usage exceeded the quota, both for time or size - +`pulsar_storage_backlog_quota_exceeded_evictions_total`, where the `quota_type` label will be either `time` or +`size`. Monitoring that counter over time will allow the user to know when a topic backlog exceeded its quota, +and if backlog eviction was chosen as action, then it happened, and how many times. + +Some users may want the backlog quota check to happen more frequently, and as a consequence, the backlog age +metric more frequently updated. They can modify `backlogQuotaCheckIntervalInSeconds` configuration key, but without +knowing how long this check takes, it will be hard for them. Hence, we will add the metric +`pulsar_storage_backlog_quota_check_duration_seconds` which will be of histogram type. + +# Detailed Design + +## Public-facing Changes + +### Public API +Adding the following to the response of topic stats, of both `{tenant}/{namespace}/{topic}/stats` +and `{tenant}/{namespace}/{topic}/partitioned-stats`: + +* `backlogQuotaLimitSize` - the size in bytes of the topic backlog quota +* `backlogQuotaLimitTime` - the topic backlog age quota, in seconds. +* `oldestBacklogMessageAgeSeconds` - the age of the oldest unacknowledged (i.e. backlog) message, measured by + the time elapsed from its published time, in seconds. This value is recorded every backlog quota check + interval, hence it represents the value seen in the last check. +* `oldestBacklogMessageSubscriptionName` - the name of the subscription containing the oldest unacknowledged message. + This value is recorded every backlog quota check interval, hence it represents the value seen in the last check. + + +### Metrics + +| Name | Description | Attributes | Units | +|----------------------------------------------------------------|-----------------------------------------------------------------------------------------------------|--------------------------------------------------------|---------| +| `pulsar_storage_backlog_age_seconds` | Gauge. The age of the oldest unacknowledged message (backlog) | cluster, namespace, topic | seconds | +| `pulsar_storage_backlog_quota_exceeded_evictions_total` | Counter. The number of times a backlog was evicted since it has exceeded its quota | cluster, namespace, topic, quota_type = (time \| size) | | +| `pulsar_storage_backlog_quota_check_duration_seconds` | Histogram. The duration of the backlog quota check process. | cluster | seconds | +| `pulsar_broker_storage_backlog_quota_exceeded_evictions_total` | Counter. The number of times a backlog was evicted since it has exceeded its quota, in broker level | cluster, quota_type = (time \| size) | | + +* Since `pulsar_storage_backlog_age_seconds` can not be aggregated, with proper meaning, to a namespace-level, it will + not be included as a metric when configuration key `exposeTopicLevelMetricsInPrometheus` is set to false. +* `pulsar_storage_backlog_quota_exceeded_evictions_total` will be included as a metric also in namespace aggregation. + +# Alternatives + +One alternative is to separate the backlog quota check into 2 separate processes, running in their own frequency: +1. Check backlog quota exceeded for all persistent topics. The result will be marked in memory. + If precise time backlog quota was configured then this will the I/O cost as described before. +2. Evict messages for those topics marked. + +This *may* enable more frequent updates to the backlog age metric making it more fresh, but the cost associated with it +might be high, since it might result in more frequent I/O calls, especially with many topics. +Another disadvantage is that it makes the backlog check and eviction more complex. + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/xv33xjjzc3t2n06ynz2gmcd4s06ckrqh +* Mailing List voting thread: https://lists.apache.org/thread/x2ypnft3x5jdyyxbwgvzxgcw20o44vps diff --git a/pom.xml b/pom.xml index 2ce1407ecfd81..ed53e11b614d5 100644 --- a/pom.xml +++ b/pom.xml @@ -140,8 +140,8 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.100.Final - 0.0.21.Final + 4.1.104.Final + 0.0.24.Final 9.4.53.v20231009 2.5.2 2.34 @@ -154,7 +154,7 @@ flexible messaging model and an intuitive client API. 2.18.0 1.75 1.0.6 - 1.0.2.3 + 1.0.2.4 2.14.2 0.10.2 1.6.2 @@ -2247,6 +2247,8 @@ flexible messaging model and an intuitive client API. distribution docker tests + + microbench @@ -2504,6 +2506,25 @@ flexible messaging model and an intuitive client API. pulsar-sql + + + microbench + + microbench + + + true + true + true + true + true + true + none + true + true + true + + diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 06d26ad680873..4f2d56fc07ea7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -1757,7 +1757,7 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, @FieldContext( category = CATEGORY_STORAGE_BK, doc = "Enable/disable reordering read sequence on reading entries") - private boolean bookkeeperClientReorderReadSequenceEnabled = false; + private boolean bookkeeperClientReorderReadSequenceEnabled = true; @FieldContext( category = CATEGORY_STORAGE_BK, required = false, @@ -2661,6 +2661,27 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, ) private boolean loadBalancerSheddingBundlesWithPoliciesEnabled = false; + @FieldContext( + category = CATEGORY_LOAD_BALANCER, + doc = "Time to wait before fixing any stuck in-flight service unit states. " + + "The leader monitor fixes any in-flight service unit(bundle) states " + + "by reassigning the ownerships if stuck too long, longer than this period." + + "(only used in load balancer extension logics)" + ) + private long loadBalancerInFlightServiceUnitStateWaitingTimeInMillis = 30 * 1000; + + @FieldContext( + category = CATEGORY_LOAD_BALANCER, + doc = "Interval between service unit state monitor checks. " + + "The service unit(bundle) state channel is periodically monitored" + + " by the leader broker at this interval" + + " to fix any orphan bundle ownerships, stuck in-flight states, and other cleanup jobs." + + "`loadBalancerServiceUnitStateTombstoneDelayTimeInSeconds` * 1000 must be bigger than " + + "`loadBalancerInFlightServiceUnitStateWaitingTimeInMillis`." + + "(only used in load balancer extension logics)" + ) + private long loadBalancerServiceUnitStateMonitorIntervalInSeconds = 60; + /**** --- Replication. --- ****/ @FieldContext( category = CATEGORY_REPLICATION, @@ -2776,7 +2797,7 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, category = CATEGORY_SERVER, doc = "Whether retain null-key message during topic compaction." ) - private boolean topicCompactionRemainNullKey = false; + private boolean topicCompactionRetainNullKey = false; @FieldContext( category = CATEGORY_SERVER, diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index ece22fe223b97..acb6fce9b92e4 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -357,6 +357,9 @@ private CompletableFuture updateSubscriptionPermissionAsync(NamespaceName policies.auth_policies.getSubscriptionAuthentication().get(subscriptionName); if (subscriptionAuth != null) { subscriptionAuth.removeAll(roles); + if (subscriptionAuth.isEmpty()) { + policies.auth_policies.getSubscriptionAuthentication().remove(subscriptionName); + } } else { log.info("[{}] Couldn't find role {} while revoking for sub = {}", namespace, roles, subscriptionName); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadBalanceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadBalanceResources.java index e13efefee0b69..57a2d16e4e89c 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadBalanceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadBalanceResources.java @@ -22,6 +22,7 @@ import java.util.concurrent.CompletableFuture; import lombok.Getter; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.policies.data.ResourceQuota; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.policies.data.loadbalancer.BundleData; import org.apache.pulsar.policies.data.loadbalancer.TimeAverageBrokerData; @@ -30,13 +31,16 @@ public class LoadBalanceResources { public static final String BUNDLE_DATA_BASE_PATH = "/loadbalance/bundle-data"; public static final String BROKER_TIME_AVERAGE_BASE_PATH = "/loadbalance/broker-time-average"; + public static final String RESOURCE_QUOTA_BASE_PATH = "/loadbalance/resource-quota"; private final BundleDataResources bundleDataResources; private final BrokerTimeAverageDataResources brokerTimeAverageDataResources; + private final QuotaResources quotaResources; public LoadBalanceResources(MetadataStore store, int operationTimeoutSec) { bundleDataResources = new BundleDataResources(store, operationTimeoutSec); brokerTimeAverageDataResources = new BrokerTimeAverageDataResources(store, operationTimeoutSec); + quotaResources = new QuotaResources(store, operationTimeoutSec); } public static class BundleDataResources extends BaseResources { @@ -92,4 +96,38 @@ private String getTimeAverageBrokerDataPath(final String brokerLookupAddress) { return BROKER_TIME_AVERAGE_BASE_PATH + "/" + brokerLookupAddress; } } + + public static class QuotaResources extends BaseResources { + public QuotaResources(MetadataStore store, int operationTimeoutSec) { + super(store, ResourceQuota.class, operationTimeoutSec); + } + + public CompletableFuture> getQuota(String bundle) { + return getAsync(getBundleQuotaPath(bundle)); + } + + public CompletableFuture> getDefaultQuota() { + return getAsync(getDefaultBundleQuotaPath()); + } + + public CompletableFuture setWithCreateQuotaAsync(String bundle, ResourceQuota quota) { + return setWithCreateAsync(getBundleQuotaPath(bundle), __ -> quota); + } + + public CompletableFuture setWithCreateDefaultQuotaAsync(ResourceQuota quota) { + return setWithCreateAsync(getDefaultBundleQuotaPath(), __ -> quota); + } + + public CompletableFuture deleteQuota(String bundle) { + return deleteAsync(getBundleQuotaPath(bundle)); + } + + private String getBundleQuotaPath(String bundle) { + return String.format("%s/%s", RESOURCE_QUOTA_BASE_PATH, bundle); + } + + private String getDefaultBundleQuotaPath() { + return getBundleQuotaPath("default"); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 02ea3bd713569..a04a4c137ccbc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -91,6 +91,8 @@ import org.apache.pulsar.broker.lookup.v1.TopicLookup; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.protocol.ProtocolHandlers; +import org.apache.pulsar.broker.qos.DefaultMonotonicSnapshotClock; +import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; import org.apache.pulsar.broker.resourcegroup.ResourceGroupService; import org.apache.pulsar.broker.resourcegroup.ResourceUsageTopicTransportManager; import org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager; @@ -191,6 +193,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private static final Logger LOG = LoggerFactory.getLogger(PulsarService.class); private static final double GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT = 0.5d; + private static final int DEFAULT_MONOTONIC_CLOCK_GRANULARITY_MILLIS = 8; private final ServiceConfiguration config; private NamespaceService nsService = null; private ManagedLedgerStorage managedLedgerClientFactory = null; @@ -271,6 +274,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private TransactionPendingAckStoreProvider transactionPendingAckStoreProvider; private final ExecutorProvider transactionExecutorProvider; + private final DefaultMonotonicSnapshotClock monotonicSnapshotClock; public enum State { Init, Started, Closing, Closed @@ -348,6 +352,9 @@ public PulsarService(ServiceConfiguration config, // here in the constructor we don't have the offloader scheduler yet this.offloaderStats = LedgerOffloaderStats.create(false, false, null, 0); + + this.monotonicSnapshotClock = new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos( + DEFAULT_MONOTONIC_CLOCK_GRANULARITY_MILLIS), System::nanoTime); } public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchronizer synchronizer) @@ -596,6 +603,7 @@ public CompletableFuture closeAsync() { brokerClientSharedInternalExecutorProvider.shutdownNow(); brokerClientSharedScheduledExecutorProvider.shutdownNow(); brokerClientSharedTimer.stop(); + monotonicSnapshotClock.close(); asyncCloseFutures.add(EventLoopUtil.shutdownGracefully(ioEventLoopGroup)); @@ -1777,6 +1785,10 @@ public Optional getBrokerListenPortTls() { return brokerService.getListenPortTls(); } + public MonotonicSnapshotClock getMonotonicSnapshotClock() { + return monotonicSnapshotClock; + } + public static WorkerConfig initializeWorkerConfigFromBrokerConfig(ServiceConfiguration brokerConfig, String workerConfigFile) throws IOException { WorkerConfig workerConfig = WorkerConfig.load(workerConfigFile); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index b8743933098fe..2f064d7b37720 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -61,7 +61,6 @@ import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; -import org.apache.pulsar.common.policies.data.ClusterPolicies; import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.ClusterPoliciesImpl; import org.apache.pulsar.common.policies.data.FailureDomainImpl; @@ -262,26 +261,29 @@ public void updateCluster( @ApiResponse(code = 404, message = "Cluster doesn't exist."), @ApiResponse(code = 500, message = "Internal server error.") }) - public ClusterPolicies getClusterMigration( + public void getClusterMigration( + @Suspended AsyncResponse asyncResponse, @ApiParam( value = "The cluster name", required = true ) - @PathParam("cluster") String cluster - ) { - validateSuperUserAccess(); - - try { - return clusterResources().getClusterPoliciesResources().getClusterPolicies(cluster) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster does not exist")); - } catch (Exception e) { - log.error("[{}] Failed to get cluster {}", clientAppId(), cluster, e); - if (e instanceof RestException) { - throw (RestException) e; - } else { - throw new RestException(e); - } - } + @PathParam("cluster") String cluster) { + validateSuperUserAccessAsync() + .thenCompose(__ -> clusterResources().getClusterPoliciesResources().getClusterPoliciesAsync(cluster)) + .thenAccept(policies -> { + asyncResponse.resume( + policies.orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster does not exist"))); + }) + .exceptionally(ex -> { + log.error("[{}] Failed to get cluster {} migration", clientAppId(), cluster, ex); + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + if (realCause instanceof MetadataStoreException.NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Cluster does not exist")); + return null; + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } @POST diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index c5174991298d4..caaff010439d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -1178,7 +1178,8 @@ protected void internalSetPublishRate(PublishRate maxPublishMessageRate) { protected CompletableFuture internalSetPublishRateAsync(PublishRate maxPublishMessageRate) { log.info("[{}] Set namespace publish-rate {}/{}", clientAppId(), namespaceName, maxPublishMessageRate); - return validateSuperUserAccessAsync().thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.publishMaxMessageRate.put(pulsar().getConfiguration().getClusterName(), maxPublishMessageRate); log.info("[{}] Successfully updated the publish_max_message_rate for cluster on namespace {}", clientAppId(), namespaceName); @@ -1207,7 +1208,8 @@ protected void internalRemovePublishRate() { protected CompletableFuture internalRemovePublishRateAsync() { log.info("[{}] Remove namespace publish-rate {}/{}", clientAppId(), namespaceName, topicName); - return validateSuperUserAccessAsync().thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { if (policies.publishMaxMessageRate != null) { policies.publishMaxMessageRate.remove(pulsar().getConfiguration().getClusterName()); } @@ -1227,7 +1229,8 @@ protected CompletableFuture internalGetPublishRateAsync() { @SuppressWarnings("deprecation") protected CompletableFuture internalSetTopicDispatchRateAsync(DispatchRateImpl dispatchRate) { log.info("[{}] Set namespace dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); - return validateSuperUserAccessAsync().thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.topicDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); policies.clusterDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); log.info("[{}] Successfully updated the dispatchRate for cluster on namespace {}", clientAppId(), @@ -1237,7 +1240,8 @@ protected CompletableFuture internalSetTopicDispatchRateAsync(DispatchRate } protected CompletableFuture internalDeleteTopicDispatchRateAsync() { - return validateSuperUserAccessAsync().thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.topicDispatchRate.remove(pulsar().getConfiguration().getClusterName()); policies.clusterDispatchRate.remove(pulsar().getConfiguration().getClusterName()); log.info("[{}] Successfully delete the dispatchRate for cluster on namespace {}", clientAppId(), @@ -1254,7 +1258,7 @@ protected CompletableFuture internalGetTopicDispatchRateAsync() { } protected CompletableFuture internalSetSubscriptionDispatchRateAsync(DispatchRateImpl dispatchRate) { - return validateSuperUserAccessAsync() + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.subscriptionDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); log.info("[{}] Successfully updated the subscriptionDispatchRate for cluster on namespace {}", @@ -1264,7 +1268,7 @@ protected CompletableFuture internalSetSubscriptionDispatchRateAsync(Dispa } protected CompletableFuture internalDeleteSubscriptionDispatchRateAsync() { - return validateSuperUserAccessAsync() + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.subscriptionDispatchRate.remove(pulsar().getConfiguration().getClusterName()); log.info("[{}] Successfully delete the subscriptionDispatchRate for cluster on namespace {}", @@ -1282,7 +1286,8 @@ protected CompletableFuture internalGetSubscriptionDispatchRateAsy protected CompletableFuture internalSetSubscribeRateAsync(SubscribeRate subscribeRate) { log.info("[{}] Set namespace subscribe-rate {}/{}", clientAppId(), namespaceName, subscribeRate); - return validateSuperUserAccessAsync().thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.clusterSubscribeRate.put(pulsar().getConfiguration().getClusterName(), subscribeRate); log.info("[{}] Successfully updated the subscribeRate for cluster on namespace {}", clientAppId(), namespaceName); @@ -1291,7 +1296,8 @@ protected CompletableFuture internalSetSubscribeRateAsync(SubscribeRate su } protected CompletableFuture internalDeleteSubscribeRateAsync() { - return validateSuperUserAccessAsync().thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { policies.clusterSubscribeRate.remove(pulsar().getConfiguration().getClusterName()); log.info("[{}] Successfully delete the subscribeRate for cluster on namespace {}", clientAppId(), namespaceName); @@ -1624,7 +1630,7 @@ protected Boolean internalGetEncryptionRequired() { } protected void internalSetInactiveTopic(InactiveTopicPolicies inactiveTopicPolicies) { - validateSuperUserAccess(); + validateNamespacePolicyOperation(namespaceName, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); internalSetPolicies("inactive_topic_policies", inactiveTopicPolicies); } @@ -2010,7 +2016,7 @@ protected void internalSetMaxUnackedMessagesPerConsumer(Integer maxUnackedMessag } protected void internalSetMaxSubscriptionsPerTopic(Integer maxSubscriptionsPerTopic){ - validateSuperUserAccess(); + validateNamespacePolicyOperationAsync(namespaceName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); if (maxSubscriptionsPerTopic != null && maxSubscriptionsPerTopic < 0) { throw new RestException(Status.PRECONDITION_FAILED, @@ -2518,7 +2524,7 @@ protected CompletableFuture internalSetEntryFiltersPerTopicAsync(EntryFilt * Notion: don't re-use this logic. */ protected void internalSetReplicatorDispatchRate(AsyncResponse asyncResponse, DispatchRateImpl dispatchRate) { - validateSuperUserAccessAsync() + validateNamespacePolicyOperationAsync(namespaceName, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE) .thenAccept(__ -> { log.info("[{}] Set namespace replicator dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); @@ -2563,7 +2569,7 @@ protected void internalGetReplicatorDispatchRate(AsyncResponse asyncResponse) { * Notion: don't re-use this logic. */ protected void internalRemoveReplicatorDispatchRate(AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateNamespacePolicyOperationAsync(namespaceName, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE) .thenCompose(__ -> namespaceResources().setPoliciesAsync(namespaceName, policies -> { String clusterName = pulsar().getConfiguration().getClusterName(); policies.replicatorDispatchRate.remove(clusterName); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 9d518ed952227..787ffe397de05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -188,19 +188,6 @@ protected CompletableFuture> internalGetListAsync(Optional ); } - protected CompletableFuture> internalGetListAsync() { - return validateNamespaceOperationAsync(namespaceName, NamespaceOperation.GET_TOPICS) - .thenCompose(__ -> namespaceResources().namespaceExistsAsync(namespaceName)) - .thenAccept(exists -> { - if (!exists) { - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } - }) - .thenCompose(__ -> topicResources().listPersistentTopicsAsync(namespaceName)) - .thenApply(topics -> topics.stream().filter(topic -> - !isTransactionInternalName(TopicName.get(topic))).collect(Collectors.toList())); - } - protected CompletableFuture> internalGetPartitionedTopicListAsync() { return validateNamespaceOperationAsync(namespaceName, NamespaceOperation.GET_TOPICS) .thenCompose(__ -> namespaceResources().namespaceExistsAsync(namespaceName)) @@ -4504,54 +4491,6 @@ protected CompletableFuture internalValidateClientVersionAsync() { return CompletableFuture.completedFuture(null); } - /** - * Validate update of number of partition for partitioned topic. - * If there's already non partition topic with same name and contains partition suffix "-partition-" - * followed by numeric value X then the new number of partition of that partitioned topic can not be greater - * than that X else that non partition topic will essentially be overwritten and cause unexpected consequence. - * - * @param topicName - */ - private CompletableFuture validatePartitionTopicUpdateAsync(String topicName, int numberOfPartition) { - return internalGetListAsync().thenCompose(existingTopicList -> { - TopicName partitionTopicName = TopicName.get(domain(), namespaceName, topicName); - String prefix = partitionTopicName.getPartitionedTopicName() + PARTITIONED_TOPIC_SUFFIX; - return getPartitionedTopicMetadataAsync(partitionTopicName, false, false) - .thenAccept(metadata -> { - int oldPartition = metadata.partitions; - for (String existingTopicName : existingTopicList) { - if (existingTopicName.startsWith(prefix)) { - try { - long suffix = Long.parseLong(existingTopicName.substring( - existingTopicName.indexOf(PARTITIONED_TOPIC_SUFFIX) - + PARTITIONED_TOPIC_SUFFIX.length())); - // Skip partition of partitioned topic by making sure - // the numeric suffix greater than old partition number. - if (suffix >= oldPartition && suffix <= (long) numberOfPartition) { - log.warn( - "[{}] Already have non partition topic {} which contains partition" - + " suffix '-partition-' and end with numeric value smaller" - + " than the new number of partition. Update of partitioned" - + " topic {} could cause conflict.", - clientAppId(), - existingTopicName, topicName); - throw new RestException(Status.PRECONDITION_FAILED, - "Already have non partition topic " + existingTopicName - + " which contains partition suffix '-partition-' " - + "and end with numeric value and end with numeric value" - + " smaller than the new number of partition. Update of" - + " partitioned topic " + topicName + " could cause conflict."); - } - } catch (NumberFormatException e) { - // Do nothing, if value after partition suffix is not pure numeric value, - // as it can't conflict with internal created partitioned topic's name. - } - } - } - }); - }); - } - /** * Validate non partition topic name, * Validation will fail and throw RestException if diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index 8c2195a9b9bf3..1e4ac7d9f5f2c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -466,7 +466,8 @@ public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); - validateAdminAccessForTenantAsync(tenant) + validateNamespacePolicyOperationAsync(namespaceName, PolicyName.SUBSCRIPTION_EXPIRATION_TIME, + PolicyOperation.READ) .thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) .thenAccept(policies -> asyncResponse.resume(policies.subscription_expiration_time_minutes)) .exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/cache/BundlesQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/cache/BundlesQuotas.java index d70520a09f3ce..d61fa0b0c81d5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/cache/BundlesQuotas.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/cache/BundlesQuotas.java @@ -19,18 +19,13 @@ package org.apache.pulsar.broker.cache; import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.resources.LoadBalanceResources; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.policies.data.ResourceQuota; -import org.apache.pulsar.metadata.api.MetadataCache; -import org.apache.pulsar.metadata.api.MetadataStore; public class BundlesQuotas { - - // Root path for resource-quota - private static final String RESOURCE_QUOTA_ROOT = "/loadbalance/resource-quota"; - private static final String DEFAULT_RESOURCE_QUOTA_PATH = RESOURCE_QUOTA_ROOT + "/default"; - - private final MetadataCache resourceQuotaCache; + LoadBalanceResources loadBalanceResources; // Default initial quota static final ResourceQuota INITIAL_QUOTA = new ResourceQuota(); @@ -44,24 +39,21 @@ public class BundlesQuotas { INITIAL_QUOTA.setDynamic(true); // allow dynamically re-calculating } - public BundlesQuotas(MetadataStore localStore) { - this.resourceQuotaCache = localStore.getMetadataCache(ResourceQuota.class); + public BundlesQuotas(PulsarService pulsar) { + loadBalanceResources = pulsar.getPulsarResources().getLoadBalanceResources(); } public CompletableFuture setDefaultResourceQuota(ResourceQuota quota) { - return resourceQuotaCache.readModifyUpdateOrCreate(DEFAULT_RESOURCE_QUOTA_PATH, __ -> quota) - .thenApply(__ -> null); + return loadBalanceResources.getQuotaResources().setWithCreateDefaultQuotaAsync(quota); } public CompletableFuture getDefaultResourceQuota() { - return resourceQuotaCache.get(DEFAULT_RESOURCE_QUOTA_PATH) + return loadBalanceResources.getQuotaResources().getDefaultQuota() .thenApply(optResourceQuota -> optResourceQuota.orElse(INITIAL_QUOTA)); } public CompletableFuture setResourceQuota(String bundle, ResourceQuota quota) { - return resourceQuotaCache.readModifyUpdateOrCreate(RESOURCE_QUOTA_ROOT + "/" + bundle, - __ -> quota) - .thenApply(__ -> null); + return loadBalanceResources.getQuotaResources().setWithCreateQuotaAsync(bundle, quota); } public CompletableFuture setResourceQuota(NamespaceBundle bundle, ResourceQuota quota) { @@ -73,7 +65,7 @@ public CompletableFuture getResourceQuota(NamespaceBundle bundle) } public CompletableFuture getResourceQuota(String bundle) { - return resourceQuotaCache.get(RESOURCE_QUOTA_ROOT + "/" + bundle) + return loadBalanceResources.getQuotaResources().getQuota(bundle) .thenCompose(optResourceQuota -> { if (optResourceQuota.isPresent()) { return CompletableFuture.completedFuture(optResourceQuota.get()); @@ -84,7 +76,6 @@ public CompletableFuture getResourceQuota(String bundle) { } public CompletableFuture resetResourceQuota(NamespaceBundle bundle) { - return resourceQuotaCache.delete(RESOURCE_QUOTA_ROOT + "/" + bundle.toString()); + return loadBalanceResources.getQuotaResources().deleteQuota(bundle.toString()); } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index d7a3e80f086d2..f98c9e000f150 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -140,6 +140,8 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException { ManagedCursor cursor = this.lastMutableBucket.getCursor(); Map cursorProperties = cursor.getCursorProperties(); if (MapUtils.isEmpty(cursorProperties)) { + log.info("[{}] Recover delayed message index bucket snapshot finish, don't find bucket snapshot", + dispatcher.getName()); return 0; } FutureUtil.Sequencer sequencer = this.lastMutableBucket.getSequencer(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java index 0de2ae92db61a..80f887d394dd9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java @@ -61,7 +61,7 @@ public void start() throws PulsarServerException { localResourceUnit = new SimpleResourceUnit(String.format("http://%s", lookupServiceAddress), new PulsarResourceDescription()); - LocalBrokerData localData = new LocalBrokerData(pulsar.getSafeWebServiceAddress(), + LocalBrokerData localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getAdvertisedListeners()); localData.setProtocols(pulsar.getProtocolDataToAdvertise()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index 921ce35b5c65e..bfdaa078f1999 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -84,7 +84,7 @@ public BrokerRegistryImpl(PulsarService pulsar) { this.listeners = new ArrayList<>(); this.brokerId = pulsar.getLookupServiceAddress(); this.brokerLookupData = new BrokerLookupData( - pulsar.getSafeWebServiceAddress(), + pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 67bab9b12ffb1..581183cf95ad3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -297,13 +297,18 @@ public static void createSystemTopic(PulsarService pulsar, String topic) throws log.info("Created topic {}.", topic); } catch (PulsarAdminException.ConflictException ex) { if (debug(pulsar.getConfiguration(), log)) { - log.info("Topic {} already exists.", topic, ex); + log.info("Topic {} already exists.", topic); } } catch (PulsarAdminException e) { throw new PulsarServerException(e); } } + private static void createSystemTopics(PulsarService pulsar) throws PulsarServerException { + createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC); + createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); + } + /** * Gets the assigned broker for the given topic. * @param pulsar PulsarService instance @@ -353,7 +358,7 @@ public void start() throws PulsarServerException { } }); }); - this.serviceUnitStateChannel = ServiceUnitStateChannelImpl.newInstance(pulsar); + this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar); this.brokerRegistry.start(); this.splitManager = new SplitManager(splitCounter); this.unloadManager = new UnloadManager(unloadCounter); @@ -370,13 +375,9 @@ public void start() throws PulsarServerException { this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies); this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper)); - createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC); - createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); - try { this.brokerLoadDataStore = LoadDataStoreFactory .create(pulsar.getClient(), BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); - this.brokerLoadDataStore.startTableView(); this.topBundlesLoadDataStore = LoadDataStoreFactory .create(pulsar.getClient(), TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); } catch (LoadDataStoreException e) { @@ -431,7 +432,6 @@ public void start() throws PulsarServerException { this.unloadScheduler = new UnloadScheduler( pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context, serviceUnitStateChannel, unloadCounter, unloadMetrics); - this.unloadScheduler.start(); this.splitScheduler = new SplitScheduler( pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); this.splitScheduler.start(); @@ -781,7 +781,7 @@ public void close() throws PulsarServerException { } } - private boolean isInternalTopic(String topic) { + public static boolean isInternalTopic(String topic) { return topic.startsWith(ServiceUnitStateChannelImpl.TOPIC) || topic.startsWith(BROKER_LOAD_DATA_STORE_TOPIC) || topic.startsWith(TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); @@ -789,74 +789,74 @@ private boolean isInternalTopic(String topic) { @VisibleForTesting void playLeader() { - if (role != Leader) { - log.info("This broker:{} is changing the role from {} to {}", - pulsar.getLookupServiceAddress(), role, Leader); - int retry = 0; - while (true) { + log.info("This broker:{} is setting the role from {} to {}", + pulsar.getLookupServiceAddress(), role, Leader); + int retry = 0; + while (!Thread.currentThread().isInterrupted()) { + try { + initWaiter.await(); + // Confirm the system topics have been created or create them if they do not exist. + // If the leader has changed, the new leader need to reset + // the local brokerService.topics (by this topic creations). + // Otherwise, the system topic existence check will fail on the leader broker. + createSystemTopics(pulsar); + brokerLoadDataStore.init(); + topBundlesLoadDataStore.init(); + unloadScheduler.start(); + serviceUnitStateChannel.scheduleOwnershipMonitor(); + break; + } catch (Throwable e) { + log.error("The broker:{} failed to set the role. Retrying {} th ...", + pulsar.getLookupServiceAddress(), ++retry, e); try { - initWaiter.await(); - serviceUnitStateChannel.scheduleOwnershipMonitor(); - topBundlesLoadDataStore.startTableView(); - unloadScheduler.start(); - break; - } catch (Throwable e) { - log.error("The broker:{} failed to change the role. Retrying {} th ...", - pulsar.getLookupServiceAddress(), ++retry, e); - try { - Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); - } catch (InterruptedException ex) { - log.warn("Interrupted while sleeping."); - } + Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); + } catch (InterruptedException ex) { + log.warn("Interrupted while sleeping."); + // preserve thread's interrupt status + Thread.currentThread().interrupt(); } } - role = Leader; - log.info("This broker:{} plays the leader now.", pulsar.getLookupServiceAddress()); } + role = Leader; + log.info("This broker:{} plays the leader now.", pulsar.getLookupServiceAddress()); // flush the load data when the leader is elected. - if (brokerLoadDataReporter != null) { - brokerLoadDataReporter.reportAsync(true); - } - if (topBundleLoadDataReporter != null) { - topBundleLoadDataReporter.reportAsync(true); - } + brokerLoadDataReporter.reportAsync(true); + topBundleLoadDataReporter.reportAsync(true); } @VisibleForTesting void playFollower() { - if (role != Follower) { - log.info("This broker:{} is changing the role from {} to {}", - pulsar.getLookupServiceAddress(), role, Follower); - int retry = 0; - while (true) { + log.info("This broker:{} is setting the role from {} to {}", + pulsar.getLookupServiceAddress(), role, Follower); + int retry = 0; + while (!Thread.currentThread().isInterrupted()) { + try { + initWaiter.await(); + unloadScheduler.close(); + serviceUnitStateChannel.cancelOwnershipMonitor(); + brokerLoadDataStore.init(); + topBundlesLoadDataStore.close(); + topBundlesLoadDataStore.startProducer(); + break; + } catch (Throwable e) { + log.error("The broker:{} failed to set the role. Retrying {} th ...", + pulsar.getLookupServiceAddress(), ++retry, e); try { - initWaiter.await(); - serviceUnitStateChannel.cancelOwnershipMonitor(); - topBundlesLoadDataStore.closeTableView(); - unloadScheduler.close(); - break; - } catch (Throwable e) { - log.error("The broker:{} failed to change the role. Retrying {} th ...", - pulsar.getLookupServiceAddress(), ++retry, e); - try { - Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); - } catch (InterruptedException ex) { - log.warn("Interrupted while sleeping."); - } + Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); + } catch (InterruptedException ex) { + log.warn("Interrupted while sleeping."); + // preserve thread's interrupt status + Thread.currentThread().interrupt(); } } - role = Follower; - log.info("This broker:{} plays a follower now.", pulsar.getLookupServiceAddress()); } + role = Follower; + log.info("This broker:{} plays a follower now.", pulsar.getLookupServiceAddress()); // flush the load data when the leader is elected. - if (brokerLoadDataReporter != null) { - brokerLoadDataReporter.reportAsync(true); - } - if (topBundleLoadDataReporter != null) { - topBundleLoadDataReporter.reportAsync(true); - } + brokerLoadDataReporter.reportAsync(true); + topBundleLoadDataReporter.reportAsync(true); } public List getMetrics() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index cff45b18ec8b7..713d98b72507e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -111,13 +111,10 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { "loadbalancer-service-unit-state").toString(); public static final CompressionType MSG_COMPRESSION_TYPE = CompressionType.ZSTD; - private static final long MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS = 30 * 1000; // 30sec - private static final int OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS = 5000; private static final int OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS = 100; private static final int OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS = 3000; public static final long VERSION_ID_INIT = 1; // initial versionId - private static final long OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS = 60; public static final long MAX_CLEAN_UP_DELAY_TIME_IN_SECS = 3 * 60; // 3 mins private static final long MIN_CLEAN_UP_DELAY_TIME_IN_SECS = 0; // 0 secs to clean immediately private static final long MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS = 10; @@ -141,7 +138,7 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private long inFlightStateWaitingTimeInMillis; private long ownershipMonitorDelayTimeInSecs; - private long semiTerminalStateWaitingTimeInMillis; + private long stateTombstoneDelayTimeInMillis; private long maxCleanupDelayTimeInSecs; private long minCleanupDelayTimeInSecs; // cleanup metrics @@ -200,18 +197,8 @@ enum MetadataState { Unstable } - public static ServiceUnitStateChannelImpl newInstance(PulsarService pulsar) { - return new ServiceUnitStateChannelImpl(pulsar); - } - - public ServiceUnitStateChannelImpl(PulsarService pulsar) { - this(pulsar, MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS, OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS); - } - @VisibleForTesting - public ServiceUnitStateChannelImpl(PulsarService pulsar, - long inFlightStateWaitingTimeInMillis, - long ownershipMonitorDelayTimeInSecs) { + public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.pulsar = pulsar; this.config = pulsar.getConfig(); this.lookupServiceAddress = pulsar.getLookupServiceAddress(); @@ -219,14 +206,16 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar, this.getOwnerRequests = new ConcurrentHashMap<>(); this.cleanupJobs = new ConcurrentHashMap<>(); this.stateChangeListeners = new StateChangeListeners(); - this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateTombstoneDelayTimeInSeconds() + this.stateTombstoneDelayTimeInMillis = config.getLoadBalancerServiceUnitStateTombstoneDelayTimeInSeconds() * 1000; - this.inFlightStateWaitingTimeInMillis = inFlightStateWaitingTimeInMillis; - this.ownershipMonitorDelayTimeInSecs = ownershipMonitorDelayTimeInSecs; - if (semiTerminalStateWaitingTimeInMillis < inFlightStateWaitingTimeInMillis) { + this.inFlightStateWaitingTimeInMillis = config.getLoadBalancerInFlightServiceUnitStateWaitingTimeInMillis(); + this.ownershipMonitorDelayTimeInSecs = config.getLoadBalancerServiceUnitStateMonitorIntervalInSeconds(); + if (stateTombstoneDelayTimeInMillis < inFlightStateWaitingTimeInMillis) { throw new IllegalArgumentException( - "Invalid Config: loadBalancerServiceUnitStateCleanUpDelayTimeInSeconds < " - + (MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS / 1000) + " secs"); + "Invalid Config: loadBalancerServiceUnitStateTombstoneDelayTimeInSeconds" + + stateTombstoneDelayTimeInMillis / 1000 + " secs" + + "< loadBalancerInFlightServiceUnitStateWaitingTimeInMillis" + + inFlightStateWaitingTimeInMillis + " millis"); } this.maxCleanupDelayTimeInSecs = MAX_CLEAN_UP_DELAY_TIME_IN_SECS; this.minCleanupDelayTimeInSecs = MIN_CLEAN_UP_DELAY_TIME_IN_SECS; @@ -1457,7 +1446,7 @@ protected void monitorOwnerships(List brokers) { continue; } - if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { + if (!isActiveState(state) && now - stateData.timestamp() > stateTombstoneDelayTimeInMillis) { log.info("Found semi-terminal states to tombstone" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); tombstoneAsync(serviceUnit).whenComplete((__, e) -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java index 680a36523a214..a7deeeaad8a5c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java @@ -81,9 +81,26 @@ public interface LoadDataStore extends Closeable { */ void closeTableView() throws IOException; + + /** + * Starts the data store (both producer and table view). + */ + void start() throws LoadDataStoreException; + + /** + * Inits the data store (close and start the data store). + */ + void init() throws IOException; + /** * Starts the table view. */ void startTableView() throws LoadDataStoreException; + + /** + * Starts the producer. + */ + void startProducer() throws LoadDataStoreException; + } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index a400163ebf122..ead0a7081fd37 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -37,9 +37,9 @@ */ public class TableViewLoadDataStoreImpl implements LoadDataStore { - private TableView tableView; + private volatile TableView tableView; - private final Producer producer; + private volatile Producer producer; private final PulsarClient client; @@ -50,7 +50,6 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore { public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class clazz) throws LoadDataStoreException { try { this.client = client; - this.producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); this.topic = topic; this.clazz = clazz; } catch (Exception e) { @@ -99,6 +98,12 @@ public void closeTableView() throws IOException { } } + @Override + public void start() throws LoadDataStoreException { + startProducer(); + startTableView(); + } + @Override public void startTableView() throws LoadDataStoreException { if (tableView == null) { @@ -111,14 +116,33 @@ public void startTableView() throws LoadDataStoreException { } } + @Override + public void startProducer() throws LoadDataStoreException { + if (producer == null) { + try { + producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); + } catch (PulsarClientException e) { + producer = null; + throw new LoadDataStoreException(e); + } + } + } + @Override public void close() throws IOException { if (producer != null) { producer.close(); + producer = null; } closeTableView(); } + @Override + public void init() throws IOException { + close(); + start(); + } + private void validateTableViewStart() { if (tableView == null) { throw new IllegalStateException("table view has not been started"); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index afe4d13215c45..4ecdfefbdd041 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -72,7 +72,6 @@ import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; -import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; import org.apache.pulsar.metadata.api.Notification; @@ -105,9 +104,6 @@ public class ModularLoadManagerImpl implements ModularLoadManager { // The number of effective samples to keep for observing short term data. public static final int NUM_SHORT_SAMPLES = 10; - // Path to ZNode whose children contain ResourceQuota jsons. - public static final String RESOURCE_QUOTA_ZPATH = "/loadbalance/resource-quota"; - // Set of broker candidates to reuse so that object creation is avoided. private final Set brokerCandidateCache; @@ -115,8 +111,6 @@ public class ModularLoadManagerImpl implements ModularLoadManager { private LockManager brokersData; private ResourceLock brokerDataLock; - private MetadataCache resourceQuotaCache; - // Broker host usage object used to calculate system resource usage. private BrokerHostUsage brokerHostUsage; @@ -240,7 +234,6 @@ public void initialize(final PulsarService pulsar) { this.pulsar = pulsar; this.pulsarResources = pulsar.getPulsarResources(); brokersData = pulsar.getCoordinationService().getLockManager(LocalBrokerData.class); - resourceQuotaCache = pulsar.getLocalMetadataStore().getMetadataCache(ResourceQuota.class); pulsar.getLocalMetadataStore().registerListener(this::handleDataNotification); pulsar.getLocalMetadataStore().registerSessionListener(this::handleMetadataSessionEvent); @@ -381,8 +374,8 @@ public BundleData getBundleDataOrDefault(final String bundle) { return optBundleData.get(); } - Optional optQuota = resourceQuotaCache - .get(String.format("%s/%s", RESOURCE_QUOTA_ZPATH, bundle)).join(); + Optional optQuota = pulsarResources.getLoadBalanceResources().getQuotaResources() + .getQuota(bundle).join(); if (optQuota.isPresent()) { ResourceQuota quota = optQuota.get(); bundleData = new BundleData(NUM_SHORT_SAMPLES, NUM_LONG_SAMPLES); @@ -968,14 +961,14 @@ public void start() throws PulsarServerException { // At this point, the ports will be updated with the real port number that the server was assigned Map protocolData = pulsar.getProtocolDataToAdvertise(); - lastData = new LocalBrokerData(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(), + lastData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getAdvertisedListeners()); lastData.setProtocols(protocolData); // configure broker-topic mode lastData.setPersistentTopicsEnabled(pulsar.getConfiguration().isEnablePersistentTopics()); lastData.setNonPersistentTopicsEnabled(pulsar.getConfiguration().isEnableNonPersistentTopics()); - localData = new LocalBrokerData(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(), + localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getAdvertisedListeners()); localData.setProtocols(protocolData); localData.setBrokerVersionString(pulsar.getBrokerVersion()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java index ce46bd932f10f..ee60595a485c4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java @@ -234,7 +234,7 @@ public void initialize(final PulsarService pulsar) { brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); } this.policies = new SimpleResourceAllocationPolicies(pulsar); - lastLoadReport = new LoadReport(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(), + lastLoadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); lastLoadReport.setProtocols(pulsar.getProtocolDataToAdvertise()); lastLoadReport.setPersistentTopicsEnabled(pulsar.getConfiguration().isEnablePersistentTopics()); @@ -1072,7 +1072,7 @@ public LoadReport generateLoadReport() throws Exception { private LoadReport generateLoadReportForcefully() throws Exception { synchronized (bundleGainsCache) { try { - LoadReport loadReport = new LoadReport(pulsar.getSafeWebServiceAddress(), + LoadReport loadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); loadReport.setProtocols(pulsar.getProtocolDataToAdvertise()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java index 1d984409fe7e2..c4a39cd0d4455 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java @@ -132,10 +132,10 @@ protected CompletableFuture internalLookupTopicAsync(final TopicName pulsar().getBrokerService().getLookupRequestSemaphore().release(); return result.getLookupData(); } - }).exceptionally(ex->{ - pulsar().getBrokerService().getLookupRequestSemaphore().release(); - throw FutureUtil.wrapToCompletionException(ex); }); + }).exceptionally(ex -> { + pulsar().getBrokerService().getLookupRequestSemaphore().release(); + throw FutureUtil.wrapToCompletionException(ex); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 0d35e7cad693b..4a54d4e090852 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -1351,42 +1351,40 @@ public CompletableFuture> getOwnedTopicListForNamespaceBundle(Names } public CompletableFuture checkTopicExists(TopicName topic) { - if (topic.isPersistent()) { - if (topic.isPartitioned()) { - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) - .thenCompose(metadata -> { - // Allow creating the non-partitioned persistent topic that name includes `-partition-` - if (metadata.partitions == 0 - || topic.getPartitionIndex() < metadata.partitions) { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } - return CompletableFuture.completedFuture(false); - }); - } else { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } + CompletableFuture future; + // If the topic is persistent and the name includes `-partition-`, find the topic from the managed/ledger. + if (topic.isPersistent() && topic.isPartitioned()) { + future = pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); } else { - if (topic.isPartitioned()) { - final TopicName partitionedTopicName = TopicName.get(topic.getPartitionedTopicName()); - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(partitionedTopicName) - .thenApply((metadata) -> topic.getPartitionIndex() < metadata.partitions); - } else { - // only checks and don't do any topic creating and loading. - CompletableFuture> topicFuture = - pulsar.getBrokerService().getTopics().get(topic.toString()); - if (topicFuture == null) { - return CompletableFuture.completedFuture(false); - } else { - return topicFuture.thenApply(Optional::isPresent).exceptionally(throwable -> { - LOG.warn("[{}] topicFuture completed with exception when checkTopicExists, {}", - topic, throwable.getMessage()); - return false; - }); - } - } + future = CompletableFuture.completedFuture(false); } + + return future.thenCompose(found -> { + if (found != null && found) { + return CompletableFuture.completedFuture(true); + } + + return pulsar.getBrokerService() + .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) + .thenCompose(metadata -> { + if (metadata.partitions > 0) { + return CompletableFuture.completedFuture(true); + } + + if (topic.isPersistent()) { + return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + } else { + // The non-partitioned non-persistent topic only exist in the broker topics. + CompletableFuture> nonPersistentTopicFuture = + pulsar.getBrokerService().getTopics().get(topic.toString()); + if (nonPersistentTopicFuture == null) { + return CompletableFuture.completedFuture(false); + } else { + return nonPersistentTopicFuture.thenApply(Optional::isPresent); + } + } + }); + }); } public CompletableFuture> getListOfTopics(NamespaceName namespaceName, Mode mode) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java index 86003153714cb..0033abf36c78c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java @@ -122,10 +122,10 @@ public OwnershipCache(PulsarService pulsar, NamespaceBundleFactory bundleFactory this.ownerBrokerUrl = pulsar.getBrokerServiceUrl(); this.ownerBrokerUrlTls = pulsar.getBrokerServiceUrlTls(); this.selfOwnerInfo = new NamespaceEphemeralData(ownerBrokerUrl, ownerBrokerUrlTls, - pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), false, pulsar.getAdvertisedListeners()); this.selfOwnerInfoDisabled = new NamespaceEphemeralData(ownerBrokerUrl, ownerBrokerUrlTls, - pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), true, pulsar.getAdvertisedListeners()); this.lockManager = pulsar.getCoordinationService().getLockManager(NamespaceEphemeralData.class); this.locallyAcquiredLocks = new ConcurrentHashMap<>(); @@ -336,7 +336,7 @@ public Map> getLocallyAcqu public synchronized boolean refreshSelfOwnerInfo() { this.selfOwnerInfo = new NamespaceEphemeralData(pulsar.getBrokerServiceUrl(), - pulsar.getBrokerServiceUrlTls(), pulsar.getSafeWebServiceAddress(), + pulsar.getBrokerServiceUrlTls(), pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), false, pulsar.getAdvertisedListeners()); return selfOwnerInfo.getNativeUrl() != null || selfOwnerInfo.getNativeUrlTls() != null; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java new file mode 100644 index 0000000000000..ac9a1f03e592b --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.LongAdder; + +/** + * An asynchronous token bucket algorithm implementation that is optimized for performance with highly concurrent + * use. CAS (compare-and-swap) operations are used and multiple levels of CAS fields are used to minimize contention + * when using CAS fields. The {@link LongAdder} class is used in the hot path to hold the sum of consumed tokens. + * It is eventually consistent, meaning that the tokens are not updated on every call to the "consumeTokens" method. + *

Main usage flow: + * 1. Tokens are consumed by invoking the "consumeTokens" or "consumeTokensAndCheckIfContainsTokens" methods. + * 2. The "consumeTokensAndCheckIfContainsTokens" or "containsTokens" methods return false if there are no + * tokens available, indicating a need for throttling. + * 3. In case of throttling, the application should throttle in a way that is suitable for the use case + * and then call the "calculateThrottlingDuration" method to calculate the duration of the required pause. + * 4. After the pause duration, the application should verify if there are any available tokens by invoking the + * containsTokens method. If tokens are available, the application should cease throttling. However, if tokens are + * not available, the application should maintain the throttling and recompute the throttling duration. In a + * concurrent environment, it is advisable to use a throttling queue to ensure fair distribution of resources across + * throttled connections or clients. Once the throttling duration has elapsed, the application should select the next + * connection or client from the throttling queue to unthrottle. Before unthrottling, the application should check + * for available tokens. If tokens are still not available, the application should continue with throttling and + * repeat the throttling loop. + *

This class does not produce side effects outside its own scope. It functions similarly to a stateful function, + * akin to a counter function. In essence, it is a sophisticated counter. It can serve as a foundational component for + * constructing higher-level asynchronous rate limiter implementations, which require side effects for throttling. + *

To achieve optimal performance, pass a {@link DefaultMonotonicSnapshotClock} instance as the clock . + */ +public abstract class AsyncTokenBucket { + public static final MonotonicSnapshotClock DEFAULT_SNAPSHOT_CLOCK = requestSnapshot -> System.nanoTime(); + static final long ONE_SECOND_NANOS = TimeUnit.SECONDS.toNanos(1); + // 2^24 nanoseconds is 16 milliseconds + private static final long DEFAULT_RESOLUTION_NANOS = TimeUnit.MILLISECONDS.toNanos(16); + + // The default resolution is 16 milliseconds. This means that the consumed tokens are subtracted from the + // current amount of tokens about every 16 milliseconds. This solution helps prevent a CAS loop what could cause + // extra CPU usage when a single CAS field is updated at a high rate from multiple threads. + static long defaultResolutionNanos = DEFAULT_RESOLUTION_NANOS; + + // used in tests to disable the optimization and instead use a consistent view of the tokens + public static void switchToConsistentTokensView() { + defaultResolutionNanos = 0; + } + + public static void resetToDefaultEventualConsistentTokensView() { + defaultResolutionNanos = DEFAULT_RESOLUTION_NANOS; + } + + // atomic field updaters for the volatile fields in this class + + private static final AtomicLongFieldUpdater LAST_NANOS_UPDATER = + AtomicLongFieldUpdater.newUpdater(AsyncTokenBucket.class, "lastNanos"); + + private static final AtomicLongFieldUpdater LAST_INCREMENT_UPDATER = + AtomicLongFieldUpdater.newUpdater(AsyncTokenBucket.class, "lastIncrement"); + + private static final AtomicLongFieldUpdater TOKENS_UPDATER = + AtomicLongFieldUpdater.newUpdater(AsyncTokenBucket.class, "tokens"); + + private static final AtomicLongFieldUpdater REMAINDER_NANOS_UPDATER = + AtomicLongFieldUpdater.newUpdater(AsyncTokenBucket.class, "remainderNanos"); + + /** + * This field represents the number of tokens in the bucket. It is eventually consistent, as the + * pendingConsumedTokens are subtracted from the total number of tokens at most once during each "tick" or + * "increment", when time advances according to the configured resolution. + */ + protected volatile long tokens; + /** + * This field represents the last time the tokens were updated, in nanoseconds. + * The configured clockSource is used to obtain the current nanoseconds. + * By default, a monotonic clock (System.nanoTime()) is used. + */ + private volatile long lastNanos; + /** + * This field represents the last time the tokens were updated, in increments. + */ + private volatile long lastIncrement; + /** + * As time progresses, tokens are added to the bucket. When the rate is low, significant rounding errors could + * accumulate over time if the remainder nanoseconds are not accounted for in the calculations. This field is used + * to carry forward the leftover nanoseconds in the update calculation. + */ + private volatile long remainderNanos; + + /** + * The resolution in nanoseconds. This is the amount of time that must pass before the tokens are updated. + */ + protected final long resolutionNanos; + /** + * This field is used to obtain the current monotonic clock time in nanoseconds. + */ + private final MonotonicSnapshotClock clockSource; + /** + * This field is used to hold the sum of consumed tokens that are pending to be subtracted from the total amount of + * tokens. This solution is to prevent CAS loop contention problem. pendingConsumedTokens used JVM's LongAdder + * which has a complex solution to prevent the CAS loop content problem. + */ + private final LongAdder pendingConsumedTokens = new LongAdder(); + + protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos) { + this.clockSource = clockSource; + this.resolutionNanos = resolutionNanos; + } + + public static FinalRateAsyncTokenBucketBuilder builder() { + return new FinalRateAsyncTokenBucketBuilder(); + } + + public static DynamicRateAsyncTokenBucketBuilder builderForDynamicRate() { + return new DynamicRateAsyncTokenBucketBuilder(); + } + + protected abstract long getRatePeriodNanos(); + + protected abstract long getTargetAmountOfTokensAfterThrottling(); + + /** + * Consumes tokens and possibly updates the tokens balance. New tokens are calculated and added to the current + * tokens balance each time the update takes place. The update takes place once in every interval of the configured + * resolutionNanos or when the forceUpdateTokens parameter is true. + * When the tokens balance isn't updated, the consumed tokens are added to the pendingConsumedTokens LongAdder + * counter which gets flushed the next time the tokens are updated. This makes the tokens balance + * eventually consistent. The reason for this design choice is to optimize performance by preventing CAS loop + * contention which could cause excessive CPU consumption. + * + * @param consumeTokens number of tokens to consume, can be 0 to update the tokens balance + * @param forceUpdateTokens if true, the tokens are updated even if the configured resolution hasn't passed + * @return the current number of tokens in the bucket or Long.MIN_VALUE when the number of tokens is unknown due + * to eventual consistency + */ + private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolean forceUpdateTokens) { + if (consumeTokens < 0) { + throw new IllegalArgumentException("consumeTokens must be >= 0"); + } + long currentNanos = clockSource.getTickNanos(forceUpdateTokens); + // check if the tokens should be updated immediately + if (shouldUpdateTokensImmediately(currentNanos, forceUpdateTokens)) { + // calculate the number of new tokens since the last update + long newTokens = calculateNewTokensSinceLastUpdate(currentNanos); + // calculate the total amount of tokens to consume in this update + // flush the pendingConsumedTokens by calling "sumThenReset" + long totalConsumedTokens = consumeTokens + pendingConsumedTokens.sumThenReset(); + // update the tokens and return the current token value + return TOKENS_UPDATER.updateAndGet(this, + currentTokens -> + // after adding new tokens, limit the tokens to the capacity + Math.min(currentTokens + newTokens, getCapacity()) + // subtract the consumed tokens + - totalConsumedTokens); + } else { + // eventual consistent fast path, tokens are not updated immediately + + // add the consumed tokens to the pendingConsumedTokens LongAdder counter + if (consumeTokens > 0) { + pendingConsumedTokens.add(consumeTokens); + } + + // return Long.MIN_VALUE if the current value of tokens is unknown due to the eventual consistency + return Long.MIN_VALUE; + } + } + + /** + * Check if the tokens should be updated immediately. + * + * The tokens will be updated once every resolutionNanos nanoseconds. + * This method checks if the configured resolutionNanos has passed since the last update. + * If the forceUpdateTokens is true, the tokens will be updated immediately. + * + * @param currentNanos the current monotonic clock time in nanoseconds + * @param forceUpdateTokens if true, the tokens will be updated immediately + * @return true if the tokens should be updated immediately, false otherwise + */ + private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUpdateTokens) { + long currentIncrement = resolutionNanos != 0 ? currentNanos / resolutionNanos : 0; + long currentLastIncrement = lastIncrement; + return currentIncrement == 0 + || (currentIncrement > currentLastIncrement + && LAST_INCREMENT_UPDATER.compareAndSet(this, currentLastIncrement, currentIncrement)) + || forceUpdateTokens; + } + + /** + * Calculate the number of new tokens since the last update. + * This will carry forward the remainder nanos so that a possible rounding error is eliminated. + * + * @param currentNanos the current monotonic clock time in nanoseconds + * @return the number of new tokens to add since the last update + */ + private long calculateNewTokensSinceLastUpdate(long currentNanos) { + long newTokens; + long previousLastNanos = LAST_NANOS_UPDATER.getAndSet(this, currentNanos); + if (previousLastNanos == 0) { + newTokens = 0; + } else { + long durationNanos = currentNanos - previousLastNanos + REMAINDER_NANOS_UPDATER.getAndSet(this, 0); + long currentRate = getRate(); + long currentRatePeriodNanos = getRatePeriodNanos(); + // new tokens is the amount of tokens that are created in the duration since the last update + // with the configured rate + newTokens = (durationNanos * currentRate) / currentRatePeriodNanos; + // carry forward the remainder nanos so that the rounding error is eliminated + long remainderNanos = durationNanos - ((newTokens * currentRatePeriodNanos) / currentRate); + if (remainderNanos > 0) { + REMAINDER_NANOS_UPDATER.addAndGet(this, remainderNanos); + } + } + return newTokens; + } + + /** + * Eventually consume tokens from the bucket. + * The number of tokens is eventually consistent with the configured granularity of resolutionNanos. + * + * @param consumeTokens the number of tokens to consume + */ + public void consumeTokens(long consumeTokens) { + consumeTokensAndMaybeUpdateTokensBalance(consumeTokens, false); + } + + /** + * Eventually consume tokens from the bucket and check if tokens remain available. + * The number of tokens is eventually consistent with the configured granularity of resolutionNanos. + * Therefore, the returned result is not definite. + * + * @param consumeTokens the number of tokens to consume + * @return true if there is tokens remains, false if tokens are all consumed. The answer isn't definite since the + * comparison is made with eventually consistent token value. + */ + public boolean consumeTokensAndCheckIfContainsTokens(long consumeTokens) { + long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(consumeTokens, false); + if (currentTokens > 0) { + // tokens remain in the bucket + return true; + } else if (currentTokens == Long.MIN_VALUE) { + // when currentTokens is Long.MIN_VALUE, the current tokens balance is unknown since consumed tokens + // was added to the pendingConsumedTokens LongAdder counter. In this case, assume that tokens balance + // hasn't been updated yet and calculate a best guess of the current value by substracting the consumed + // tokens from the current tokens balance + return tokens - consumeTokens > 0; + } else { + // no tokens remain in the bucket + return false; + } + } + + /** + * Returns the current token balance. When forceUpdateTokens is true, the tokens balance is updated before + * returning. If forceUpdateTokens is false, the tokens balance could be updated if the last updated happened + * more than resolutionNanos nanoseconds ago. + * + * @param forceUpdateTokens if true, the tokens balance is updated before returning + * @return the current token balance + */ + protected long tokens(boolean forceUpdateTokens) { + long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, forceUpdateTokens); + if (currentTokens != Long.MIN_VALUE) { + // when currentTokens isn't Long.MIN_VALUE, the current tokens balance is known + return currentTokens; + } else { + // return the current tokens balance, ignore the possible pendingConsumedTokens LongAdder counter + return tokens; + } + } + + /** + * Calculate the required throttling duration in nanoseconds to fill up the bucket with the minimum amount of + * tokens. + * This method shouldn't be called from the hot path since it calculates a consistent value for the tokens which + * isn't necessary on the hotpath. + */ + public long calculateThrottlingDuration() { + long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, true); + if (currentTokens == Long.MIN_VALUE) { + throw new IllegalArgumentException( + "Unexpected result from updateAndConsumeTokens with forceUpdateTokens set to true"); + } + if (currentTokens > 0) { + return 0L; + } + // currentTokens is negative, so subtracting a negative value results in adding the absolute value (-(-x) -> +x) + long needTokens = getTargetAmountOfTokensAfterThrottling() - currentTokens; + return (needTokens * getRatePeriodNanos()) / getRate(); + } + + public abstract long getCapacity(); + + /** + * Returns the current number of tokens in the bucket. + * The token balance is updated if the configured resolutionNanos has passed since the last update. + */ + public final long getTokens() { + return tokens(false); + } + + public abstract long getRate(); + + /** + * Checks if the bucket contains tokens. + * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last + * update. It's possible that the returned result is not definite since the token balance is eventually consistent. + * + * @return true if the bucket contains tokens, false otherwise + */ + public boolean containsTokens() { + return containsTokens(false); + } + + /** + * Checks if the bucket contains tokens. + * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last + * update. The token balance is also updated when forceUpdateTokens is true. + * It's possible that the returned result is not definite since the token balance is eventually consistent. + * + * @param forceUpdateTokens if true, the token balance is updated before the comparison + * @return true if the bucket contains tokens, false otherwise + */ + public boolean containsTokens(boolean forceUpdateTokens) { + return tokens(forceUpdateTokens) > 0; + } + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java new file mode 100644 index 0000000000000..ee256d5a37d64 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +// CHECKSTYLE.OFF: ClassTypeParameterName +public abstract class AsyncTokenBucketBuilder> { + protected MonotonicSnapshotClock clock = AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK; + protected long resolutionNanos = AsyncTokenBucket.defaultResolutionNanos; + + protected AsyncTokenBucketBuilder() { + } + + protected SELF self() { + return (SELF) this; + } + + public SELF clock(MonotonicSnapshotClock clock) { + this.clock = clock; + return self(); + } + + public SELF resolutionNanos(long resolutionNanos) { + this.resolutionNanos = resolutionNanos; + return self(); + } + + public abstract AsyncTokenBucket build(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java new file mode 100644 index 0000000000000..df3843921ed55 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import java.util.concurrent.TimeUnit; +import java.util.function.LongSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Default implementation of {@link MonotonicSnapshotClock}. + * + * Starts a daemon thread that updates the snapshot value periodically with a configured interval. The close method + * should be called to stop the thread. + */ +public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, AutoCloseable { + private static final Logger LOG = LoggerFactory.getLogger(DefaultMonotonicSnapshotClock.class); + private final long sleepMillis; + private final int sleepNanos; + private final LongSupplier clockSource; + private final Thread thread; + private volatile long snapshotTickNanos; + + public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource) { + if (snapshotIntervalNanos < TimeUnit.MILLISECONDS.toNanos(1)) { + throw new IllegalArgumentException("snapshotIntervalNanos must be at least 1 millisecond"); + } + this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); + this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); + this.clockSource = clockSource; + updateSnapshotTickNanos(); + thread = new Thread(this::snapshotLoop, getClass().getSimpleName() + "-update-loop"); + thread.setDaemon(true); + thread.start(); + } + + /** {@inheritDoc} */ + @Override + public long getTickNanos(boolean requestSnapshot) { + if (requestSnapshot) { + updateSnapshotTickNanos(); + } + return snapshotTickNanos; + } + + private void updateSnapshotTickNanos() { + snapshotTickNanos = clockSource.getAsLong(); + } + + private void snapshotLoop() { + try { + while (!Thread.currentThread().isInterrupted()) { + updateSnapshotTickNanos(); + try { + Thread.sleep(sleepMillis, sleepNanos); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } catch (Throwable t) { + // report unexpected error since this would be a fatal error when the clock doesn't progress anymore + // this is very unlikely to happen, but it's better to log it in any case + LOG.error("Unexpected fatal error that stopped the clock.", t); + } + } + + @Override + public void close() { + thread.interrupt(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java new file mode 100644 index 0000000000000..8edc73d1f51e3 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import java.util.function.LongSupplier; + +/** + * A subclass of {@link AsyncTokenBucket} that represents a token bucket with a dynamic rate. + * The rate and capacity of the token bucket can change over time based on the rate function and capacity factor. + */ +public class DynamicRateAsyncTokenBucket extends AsyncTokenBucket { + private final LongSupplier rateFunction; + private final LongSupplier ratePeriodNanosFunction; + private final double capacityFactor; + + private final double targetFillFactorAfterThrottling; + + protected DynamicRateAsyncTokenBucket(double capacityFactor, LongSupplier rateFunction, + MonotonicSnapshotClock clockSource, LongSupplier ratePeriodNanosFunction, + long resolutionNanos, double initialTokensFactor, + double targetFillFactorAfterThrottling) { + super(clockSource, resolutionNanos); + this.capacityFactor = capacityFactor; + this.rateFunction = rateFunction; + this.ratePeriodNanosFunction = ratePeriodNanosFunction; + this.targetFillFactorAfterThrottling = targetFillFactorAfterThrottling; + this.tokens = (long) (rateFunction.getAsLong() * initialTokensFactor); + tokens(false); + } + + @Override + protected long getRatePeriodNanos() { + return ratePeriodNanosFunction.getAsLong(); + } + + @Override + protected long getTargetAmountOfTokensAfterThrottling() { + return (long) (getRate() * targetFillFactorAfterThrottling); + } + + @Override + public long getCapacity() { + return capacityFactor == 1.0d ? getRate() : (long) (getRate() * capacityFactor); + } + + @Override + public long getRate() { + return rateFunction.getAsLong(); + } + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java new file mode 100644 index 0000000000000..22270484c72f0 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import java.util.function.LongSupplier; + +/** + * A builder class for creating instances of {@link DynamicRateAsyncTokenBucket}. + */ +public class DynamicRateAsyncTokenBucketBuilder + extends AsyncTokenBucketBuilder { + protected LongSupplier rateFunction; + protected double capacityFactor = 1.0d; + protected double initialFillFactor = 1.0d; + protected LongSupplier ratePeriodNanosFunction; + protected double targetFillFactorAfterThrottling = 0.01d; + + protected DynamicRateAsyncTokenBucketBuilder() { + } + + public DynamicRateAsyncTokenBucketBuilder rateFunction(LongSupplier rateFunction) { + this.rateFunction = rateFunction; + return this; + } + + public DynamicRateAsyncTokenBucketBuilder ratePeriodNanosFunction(LongSupplier ratePeriodNanosFunction) { + this.ratePeriodNanosFunction = ratePeriodNanosFunction; + return this; + } + + public DynamicRateAsyncTokenBucketBuilder capacityFactor(double capacityFactor) { + this.capacityFactor = capacityFactor; + return this; + } + + public DynamicRateAsyncTokenBucketBuilder initialFillFactor(double initialFillFactor) { + this.initialFillFactor = initialFillFactor; + return this; + } + + public DynamicRateAsyncTokenBucketBuilder targetFillFactorAfterThrottling( + double targetFillFactorAfterThrottling) { + this.targetFillFactorAfterThrottling = targetFillFactorAfterThrottling; + return this; + } + + @Override + public AsyncTokenBucket build() { + return new DynamicRateAsyncTokenBucket(this.capacityFactor, this.rateFunction, + this.clock, + this.ratePeriodNanosFunction, this.resolutionNanos, + this.initialFillFactor, + targetFillFactorAfterThrottling); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java new file mode 100644 index 0000000000000..627c5ee1334b2 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +/** + * A subclass of {@link AsyncTokenBucket} that represents a token bucket with a rate which is final. + * The rate and capacity of the token bucket are constant and do not change over time. + */ +class FinalRateAsyncTokenBucket extends AsyncTokenBucket { + private final long capacity; + private final long rate; + private final long ratePeriodNanos; + private final long targetAmountOfTokensAfterThrottling; + + protected FinalRateAsyncTokenBucket(long capacity, long rate, MonotonicSnapshotClock clockSource, + long ratePeriodNanos, long resolutionNanos, long initialTokens) { + super(clockSource, resolutionNanos); + this.capacity = capacity; + this.rate = rate; + this.ratePeriodNanos = ratePeriodNanos != -1 ? ratePeriodNanos : ONE_SECOND_NANOS; + // The target amount of tokens is the amount of tokens made available in the resolution duration + this.targetAmountOfTokensAfterThrottling = Math.max(this.resolutionNanos * rate / ratePeriodNanos, 1); + this.tokens = initialTokens; + tokens(false); + } + + @Override + protected final long getRatePeriodNanos() { + return ratePeriodNanos; + } + + @Override + protected final long getTargetAmountOfTokensAfterThrottling() { + return targetAmountOfTokensAfterThrottling; + } + + @Override + public final long getCapacity() { + return capacity; + } + + @Override + public final long getRate() { + return rate; + } + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java new file mode 100644 index 0000000000000..ff4ed53c6c7fa --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +/** + * A builder class for creating instances of {@link FinalRateAsyncTokenBucket}. + */ +public class FinalRateAsyncTokenBucketBuilder + extends AsyncTokenBucketBuilder { + protected Long capacity; + protected Long initialTokens; + protected Long rate; + protected long ratePeriodNanos = AsyncTokenBucket.ONE_SECOND_NANOS; + + protected FinalRateAsyncTokenBucketBuilder() { + } + + public FinalRateAsyncTokenBucketBuilder rate(long rate) { + this.rate = rate; + return this; + } + + public FinalRateAsyncTokenBucketBuilder ratePeriodNanos(long ratePeriodNanos) { + this.ratePeriodNanos = ratePeriodNanos; + return this; + } + + public FinalRateAsyncTokenBucketBuilder capacity(long capacity) { + this.capacity = capacity; + return this; + } + + public FinalRateAsyncTokenBucketBuilder initialTokens(long initialTokens) { + this.initialTokens = initialTokens; + return this; + } + + public AsyncTokenBucket build() { + return new FinalRateAsyncTokenBucket(this.capacity != null ? this.capacity : this.rate, this.rate, + this.clock, + this.ratePeriodNanos, this.resolutionNanos, + this.initialTokens != null ? this.initialTokens : this.rate + ); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/MonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/MonotonicSnapshotClock.java new file mode 100644 index 0000000000000..8f61bd5125b5f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/MonotonicSnapshotClock.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +/** + * An interface representing a clock that provides a monotonic counter in nanoseconds. + * The counter is guaranteed to be monotonic, ensuring it will always increase or remain constant, but never decrease. + * + * Monotonicity ensures the time will always progress forward, making it ideal for measuring elapsed time. + * The monotonic clock is not related to the wall-clock time and is not affected by changes to the system time. + * The tick value is only significant when compared to other values obtained from the same clock source + * and should not be used for other purposes. + * + * This interface assumes that the implementation can be implemented in a granular way. This means that the value is + * advanced in steps of a configurable resolution that snapshots the underlying high precision monotonic clock source + * value. + * This design allows for optimizations that can improve performance on platforms where obtaining the value of a + * platform monotonic clock is relatively expensive. + */ +public interface MonotonicSnapshotClock { + /** + * Retrieves the latest snapshot of the tick value of the monotonic clock in nanoseconds. + * + * When requestSnapshot is set to true, the method will snapshot the underlying high-precision monotonic clock + * source so that the latest snapshot value is as accurate as possible. This may be a relatively expensive + * compared to a non-snapshot request. + * + * When requestSnapshot is set to false, the method will return the latest snapshot value which is updated by + * either a call that requested a snapshot or by an update thread that is configured to update the snapshot value + * periodically. + * + * This method returns a value that is guaranteed to be monotonic, meaning it will always increase or remain the + * same, never decrease. The returned value is only significant when compared to other values obtained from the same + * clock source and should not be used for other purposes. + * + * @param requestSnapshot If set to true, the method will request a new snapshot from the underlying more + * high-precision monotonic clock. + * @return The current tick value of the monotonic clock in nanoseconds. + */ + long getTickNanos(boolean requestSnapshot); +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/package-info.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/package-info.java new file mode 100644 index 0000000000000..1078d86894efe --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/** + * Pulsar broker Quality of Service (QoS) related classes. + */ +package org.apache.pulsar.broker.qos; \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java index ef40a18ab08ed..0aa355eee7d33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java @@ -81,7 +81,8 @@ protected ResourceGroup(ResourceGroupService rgs, String name, this.setResourceGroupMonitoringClassFields(); this.setResourceGroupConfigParameters(rgConfig); this.setDefaultResourceUsageTransportHandlers(); - this.resourceGroupPublishLimiter = new ResourceGroupPublishLimiter(rgConfig, rgs.getPulsar().getExecutor()); + this.resourceGroupPublishLimiter = new ResourceGroupPublishLimiter(rgConfig, rgs.getPulsar() + .getMonotonicSnapshotClock()); log.info("attaching publish rate limiter {} to {} get {}", this.resourceGroupPublishLimiter, name, this.getResourceGroupPublishLimiter()); } @@ -96,7 +97,8 @@ protected ResourceGroup(ResourceGroupService rgs, String rgName, this.resourceGroupName = rgName; this.setResourceGroupMonitoringClassFields(); this.setResourceGroupConfigParameters(rgConfig); - this.resourceGroupPublishLimiter = new ResourceGroupPublishLimiter(rgConfig, rgs.getPulsar().getExecutor()); + this.resourceGroupPublishLimiter = new ResourceGroupPublishLimiter(rgConfig, rgs.getPulsar() + .getMonotonicSnapshotClock()); this.ruPublisher = rgPublisher; this.ruConsumer = rgConsumer; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java index 85e00bb2f87dc..a733db555a351 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java @@ -18,53 +18,22 @@ */ package org.apache.pulsar.broker.resourcegroup; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; -import org.apache.pulsar.broker.service.PublishRateLimiter; +import org.apache.pulsar.broker.service.PublishRateLimiterImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.ResourceGroup; -import org.apache.pulsar.common.util.RateLimitFunction; -import org.apache.pulsar.common.util.RateLimiter; -public class ResourceGroupPublishLimiter implements PublishRateLimiter, RateLimitFunction, AutoCloseable { - protected volatile long publishMaxMessageRate = 0; - protected volatile long publishMaxByteRate = 0; - protected volatile boolean publishThrottlingEnabled = false; - private volatile RateLimiter publishRateLimiterOnMessage; - private volatile RateLimiter publishRateLimiterOnByte; - private final ScheduledExecutorService scheduledExecutorService; +public class ResourceGroupPublishLimiter extends PublishRateLimiterImpl { + private volatile long publishMaxMessageRate; + private volatile long publishMaxByteRate; - ConcurrentHashMap rateLimitFunctionMap = new ConcurrentHashMap<>(); - - public ResourceGroupPublishLimiter(ResourceGroup resourceGroup, ScheduledExecutorService scheduledExecutorService) { - this.scheduledExecutorService = scheduledExecutorService; + public ResourceGroupPublishLimiter(ResourceGroup resourceGroup, MonotonicSnapshotClock monotonicSnapshotClock) { + super(monotonicSnapshotClock); update(resourceGroup); } - @Override - public void checkPublishRate() { - // No-op - } - - @Override - public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { - // No-op - } - - @Override - public boolean resetPublishCount() { - return true; - } - - @Override - public boolean isPublishRateExceeded() { - return false; - } - @Override public void update(Policies policies, String clusterName) { // No-op @@ -94,102 +63,12 @@ public void update(ResourceGroup resourceGroup) { publishRateInMsgs = resourceGroup.getPublishRateInMsgs() == null ? -1 : resourceGroup.getPublishRateInMsgs(); } - update(publishRateInMsgs, publishRateInBytes); } public void update(long publishRateInMsgs, long publishRateInBytes) { - replaceLimiters(() -> { - if (publishRateInMsgs > 0 || publishRateInBytes > 0) { - this.publishThrottlingEnabled = true; - this.publishMaxMessageRate = Math.max(publishRateInMsgs, 0); - this.publishMaxByteRate = Math.max(publishRateInBytes, 0); - if (this.publishMaxMessageRate > 0) { - publishRateLimiterOnMessage = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(publishMaxMessageRate) - .rateTime(1L) - .timeUnit(TimeUnit.SECONDS) - .rateLimitFunction(this::apply) - .build(); - } - if (this.publishMaxByteRate > 0) { - publishRateLimiterOnByte = - RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(publishMaxByteRate) - .rateTime(1L) - .timeUnit(TimeUnit.SECONDS) - .rateLimitFunction(this::apply) - .build(); - } - } else { - this.publishMaxMessageRate = 0; - this.publishMaxByteRate = 0; - this.publishThrottlingEnabled = false; - publishRateLimiterOnMessage = null; - publishRateLimiterOnByte = null; - } - }); - } - - public boolean tryAcquire(int numbers, long bytes) { - return (publishRateLimiterOnMessage == null || publishRateLimiterOnMessage.tryAcquire(numbers)) - && (publishRateLimiterOnByte == null || publishRateLimiterOnByte.tryAcquire(bytes)); - } - - public void registerRateLimitFunction(String name, RateLimitFunction func) { - rateLimitFunctionMap.put(name, func); - } - - public void unregisterRateLimitFunction(String name) { - rateLimitFunctionMap.remove(name); - } - - private void replaceLimiters(Runnable updater) { - RateLimiter previousPublishRateLimiterOnMessage = publishRateLimiterOnMessage; - publishRateLimiterOnMessage = null; - RateLimiter previousPublishRateLimiterOnByte = publishRateLimiterOnByte; - publishRateLimiterOnByte = null; - try { - if (updater != null) { - updater.run(); - } - } finally { - // Close previous limiters to prevent resource leakages. - // Delay closing of previous limiters after new ones are in place so that updating the limiter - // doesn't cause unavailability. - if (previousPublishRateLimiterOnMessage != null) { - previousPublishRateLimiterOnMessage.close(); - } - if (previousPublishRateLimiterOnByte != null) { - previousPublishRateLimiterOnByte.close(); - } - } - } - - @Override - public void close() { - // Unblock any producers, consumers waiting first. - // This needs to be done before replacing the filters to null - this.apply(); - replaceLimiters(null); - } - - @Override - public void apply() { - // Make sure that both the rate limiters are applied before opening the flood gates. - RateLimiter currentTopicPublishRateLimiterOnMessage = publishRateLimiterOnMessage; - RateLimiter currentTopicPublishRateLimiterOnByte = publishRateLimiterOnByte; - if ((currentTopicPublishRateLimiterOnMessage != null - && currentTopicPublishRateLimiterOnMessage.getAvailablePermits() <= 0) - || (currentTopicPublishRateLimiterOnByte != null - && currentTopicPublishRateLimiterOnByte.getAvailablePermits() <= 0)) { - return; - } - - for (Map.Entry entry: rateLimitFunctionMap.entrySet()) { - entry.getValue().apply(); - } + this.publishMaxMessageRate = publishRateInMsgs; + this.publishMaxByteRate = publishRateInBytes; + updateTokenBuckets(publishRateInMsgs, publishRateInBytes); } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java index d3f8eb7613a40..48419d7c45127 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java @@ -173,7 +173,6 @@ public void resourceGroupDelete(String name) throws PulsarAdminException { throw new PulsarAdminException(errMesg); } - rg.resourceGroupPublishLimiter.close(); rg.resourceGroupPublishLimiter = null; resourceGroupsMap.remove(name); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index b36389ab2dada..2f38ad67d4f30 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.compaction.Compactor; import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j @@ -174,13 +175,15 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i if (msgMetadata != null && msgMetadata.hasTxnidMostBits() && msgMetadata.hasTxnidLeastBits()) { if (Markers.isTxnMarker(msgMetadata)) { - // because consumer can receive message is smaller than maxReadPosition, - // so this marker is useless for this subscription - individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), - Collections.emptyMap()); - entries.set(i, null); - entry.release(); - continue; + if (cursor == null || !cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION)) { + // because consumer can receive message is smaller than maxReadPosition, + // so this marker is useless for this subscription + individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), + Collections.emptyMap()); + entries.set(i, null); + entry.release(); + continue; + } } else if (((PersistentTopic) subscription.getTopic()) .isTxnAborted(new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()), (PositionImpl) entry.getPosition())) { @@ -192,19 +195,26 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i } } - if (msgMetadata == null || Markers.isServerOnlyMarker(msgMetadata)) { + if (msgMetadata == null || (Markers.isServerOnlyMarker(msgMetadata))) { PositionImpl pos = (PositionImpl) entry.getPosition(); // Message metadata was corrupted or the messages was a server-only marker if (Markers.isReplicatedSubscriptionSnapshotMarker(msgMetadata)) { + final int readerIndex = metadataAndPayload.readerIndex(); processReplicatedSubscriptionSnapshot(pos, metadataAndPayload); + metadataAndPayload.readerIndex(readerIndex); } - entries.set(i, null); - entry.release(); - individualAcknowledgeMessageIfNeeded(Collections.singletonList(pos), - Collections.emptyMap()); - continue; + // Deliver marker to __compaction cursor to avoid compaction task stuck, + // and filter out them when doing topic compaction. + if (msgMetadata == null || cursor == null + || !cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION)) { + entries.set(i, null); + entry.release(); + individualAcknowledgeMessageIfNeeded(Collections.singletonList(pos), + Collections.emptyMap()); + continue; + } } else if (trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { // The message is marked for delayed delivery. Ignore for now. entries.set(i, null); @@ -315,10 +325,10 @@ protected void acquirePermitsForDeliveredMessages(Topic topic, ManagedCursor cur || (cursor != null && !cursor.isActive())) { long permits = dispatchThrottlingOnBatchMessageEnabled ? totalEntries : totalMessagesSent; topic.getBrokerDispatchRateLimiter().ifPresent(rateLimiter -> - rateLimiter.tryDispatchPermit(permits, totalBytesSent)); + rateLimiter.consumeDispatchQuota(permits, totalBytesSent)); topic.getDispatchRateLimiter().ifPresent(rateLimter -> - rateLimter.tryDispatchPermit(permits, totalBytesSent)); - getRateLimiter().ifPresent(rateLimiter -> rateLimiter.tryDispatchPermit(permits, totalBytesSent)); + rateLimter.consumeDispatchQuota(permits, totalBytesSent)); + getRateLimiter().ifPresent(rateLimiter -> rateLimiter.consumeDispatchQuota(permits, totalBytesSent)); } } @@ -356,16 +366,6 @@ public void resetCloseFuture() { protected abstract void reScheduleRead(); - protected boolean reachDispatchRateLimit(DispatchRateLimiter dispatchRateLimiter) { - if (dispatchRateLimiter.isDispatchRateLimitingEnabled()) { - if (!dispatchRateLimiter.hasMessageDispatchPermit()) { - reScheduleRead(); - return true; - } - } - return false; - } - protected Pair updateMessagesToRead(DispatchRateLimiter dispatchRateLimiter, int messagesToRead, long bytesToRead) { // update messagesToRead according to available dispatch rate limit. @@ -376,11 +376,11 @@ protected Pair updateMessagesToRead(DispatchRateLimiter dispatchR protected static Pair computeReadLimits(int messagesToRead, int availablePermitsOnMsg, long bytesToRead, long availablePermitsOnByte) { - if (availablePermitsOnMsg > 0) { + if (availablePermitsOnMsg >= 0) { messagesToRead = Math.min(messagesToRead, availablePermitsOnMsg); } - if (availablePermitsOnByte > 0) { + if (availablePermitsOnByte >= 0) { bytesToRead = Math.min(bytesToRead, availablePermitsOnByte); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java index 310354dcd3b47..4a8a805f16bf9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Objects; +import java.util.Optional; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; @@ -32,8 +33,10 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; +import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.client.impl.Murmur3Hash32; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.FutureUtil; @@ -80,8 +83,6 @@ public AbstractDispatcherSingleActiveConsumer(SubType subscriptionType, int part protected abstract void scheduleReadOnActiveConsumer(); - protected abstract void readMoreEntries(Consumer consumer); - protected abstract void cancelPendingRead(); protected void notifyActiveConsumerChanged(Consumer activeConsumer) { @@ -257,9 +258,13 @@ public synchronized boolean canUnsubscribe(Consumer consumer) { return (consumers.size() == 1) && Objects.equals(consumer, ACTIVE_CONSUMER_UPDATER.get(this)); } - public CompletableFuture close() { + @Override + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { IS_CLOSED_UPDATER.set(this, TRUE); - return disconnectAllConsumers(); + getRateLimiter().ifPresent(DispatchRateLimiter::close); + return disconnectConsumers + ? disconnectAllConsumers(false, assignedBrokerLookupData) : CompletableFuture.completedFuture(null); } public boolean isClosed() { @@ -268,15 +273,23 @@ public boolean isClosed() { /** * Disconnect all consumers on this dispatcher (server side close). This triggers channelInactive on the inbound - * handler which calls dispatcher.removeConsumer(), where the closeFuture is completed + * handler which calls dispatcher.removeConsumer(), where the closeFuture is completed. * - * @return + * @param isResetCursor + * Specifies if the cursor has been reset. + * @param assignedBrokerLookupData + * Optional target broker redirect information. Allows the consumer to quickly reconnect to a broker + * during bundle unloading. + * + * @return CompletableFuture indicating the completion of the operation. */ - public synchronized CompletableFuture disconnectAllConsumers(boolean isResetCursor) { + @Override + public synchronized CompletableFuture disconnectAllConsumers( + boolean isResetCursor, Optional assignedBrokerLookupData) { closeFuture = new CompletableFuture<>(); if (!consumers.isEmpty()) { - consumers.forEach(consumer -> consumer.disconnect(isResetCursor)); + consumers.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); cancelPendingRead(); } else { // no consumer connected, complete disconnect immediately diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index bc2b358200c87..837f073b00dba 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -118,9 +118,7 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener> entryFilters; + protected volatile boolean transferring = false; + private volatile List activeRateLimiters; public AbstractTopic(String topic, BrokerService brokerService) { this.topic = topic; @@ -171,6 +171,8 @@ public AbstractTopic(String topic, BrokerService brokerService) { this.lastActive = System.nanoTime(); this.preciseTopicPublishRateLimitingEnable = config.isPreciseTopicPublishRateLimiterEnable(); + topicPublishRateLimiter = new PublishRateLimiterImpl(brokerService.getPulsar().getMonotonicSnapshotClock()); + updateActiveRateLimiters(); } public SubscribeRate getSubscribeRate() { @@ -564,16 +566,6 @@ protected Consumer getActiveConsumer(Subscription subscription) { return null; } - @Override - public void disableCnxAutoRead() { - producers.values().forEach(producer -> producer.getCnx().disableCnxAutoRead()); - } - - @Override - public void enableCnxAutoRead() { - producers.values().forEach(producer -> producer.getCnx().enableCnxAutoRead()); - } - protected boolean hasLocalProducers() { if (producers.isEmpty()) { return false; @@ -897,57 +889,35 @@ public long increasePublishLimitedTimes() { .register(); @Override - public void checkTopicPublishThrottlingRate() { - this.topicPublishRateLimiter.checkPublishRate(); - } + public void incrementPublishCount(Producer producer, int numOfMessages, long msgSizeInBytes) { + handlePublishThrottling(producer, numOfMessages, msgSizeInBytes); - @Override - public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { - // increase topic publish rate limiter - this.topicPublishRateLimiter.incrementPublishCount(numOfMessages, msgSizeInBytes); - // increase broker publish rate limiter - getBrokerPublishRateLimiter().incrementPublishCount(numOfMessages, msgSizeInBytes); // increase counters bytesInCounter.add(msgSizeInBytes); msgInCounter.add(numOfMessages); } - @Override - public void resetTopicPublishCountAndEnableReadIfRequired() { - // broker rate not exceeded. and completed topic limiter reset. - if (!getBrokerPublishRateLimiter().isPublishRateExceeded() && topicPublishRateLimiter.resetPublishCount()) { - enableProducerReadForPublishRateLimiting(); + private void handlePublishThrottling(Producer producer, int numOfMessages, long msgSizeInBytes) { + // consume tokens from rate limiters and possibly throttle the connection that published the message + // if it's publishing too fast. Each connection will be throttled lazily when they publish messages. + for (PublishRateLimiter rateLimiter : activeRateLimiters) { + rateLimiter.handlePublishThrottling(producer, numOfMessages, msgSizeInBytes); } } - public void updateDispatchRateLimiter() { - } - - @Override - public void resetBrokerPublishCountAndEnableReadIfRequired(boolean doneBrokerReset) { - // topic rate not exceeded, and completed broker limiter reset. - if (!topicPublishRateLimiter.isPublishRateExceeded() && doneBrokerReset) { - enableProducerReadForPublishRateLimiting(); + private void updateActiveRateLimiters() { + List updatedRateLimiters = new ArrayList<>(); + updatedRateLimiters.add(this.topicPublishRateLimiter); + updatedRateLimiters.add(getBrokerPublishRateLimiter()); + if (isResourceGroupRateLimitingEnabled()) { + updatedRateLimiters.add(resourceGroupPublishLimiter); } + activeRateLimiters = updatedRateLimiters.stream().filter(Objects::nonNull).toList(); } - /** - * it sets cnx auto-readable if producer's cnx is disabled due to publish-throttling. - */ - protected void enableProducerReadForPublishRateLimiting() { - if (producers != null) { - producers.values().forEach(producer -> { - producer.getCnx().cancelPublishRateLimiting(); - producer.getCnx().enableCnxAutoRead(); - }); - } + public void updateDispatchRateLimiter() { } - protected void disableProducerRead() { - if (producers != null) { - producers.values().forEach(producer -> producer.getCnx().disableCnxAutoRead()); - } - } protected void checkTopicFenced() throws BrokerServiceException { if (isFenced) { @@ -956,11 +926,6 @@ protected void checkTopicFenced() throws BrokerServiceException { } } - @Override - public boolean isFenced() { - return isFenced; - } - protected CompletableFuture internalAddProducer(Producer producer) { if (isProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); @@ -1103,35 +1068,6 @@ public long currentUsageCount() { return usageCount; } - @Override - public boolean isPublishRateExceeded() { - // either topic or broker publish rate exceeded. - return this.topicPublishRateLimiter.isPublishRateExceeded() - || getBrokerPublishRateLimiter().isPublishRateExceeded(); - } - - @Override - public boolean isResourceGroupPublishRateExceeded(int numMessages, int bytes) { - return this.resourceGroupRateLimitingEnabled - && !this.resourceGroupPublishLimiter.tryAcquire(numMessages, bytes); - } - - @Override - public boolean isResourceGroupRateLimitingEnabled() { - return this.resourceGroupRateLimitingEnabled; - } - - @Override - public boolean isTopicPublishRateExceeded(int numberMessages, int bytes) { - // whether topic publish rate exceed if precise rate limit is enable - return preciseTopicPublishRateLimitingEnable && !this.topicPublishRateLimiter.tryAcquire(numberMessages, bytes); - } - - @Override - public boolean isBrokerPublishRateExceeded() { - // whether broker publish rate exceed - return getBrokerPublishRateLimiter().isPublishRateExceeded(); - } public PublishRateLimiter getTopicPublishRateLimiter() { return topicPublishRateLimiter; @@ -1173,19 +1109,15 @@ public void updateResourceGroupLimiter(@Nonnull Policies namespacePolicies) { if (resourceGroup != null) { this.resourceGroupRateLimitingEnabled = true; this.resourceGroupPublishLimiter = resourceGroup.getResourceGroupPublishLimiter(); - this.resourceGroupPublishLimiter.registerRateLimitFunction(this.getName(), this::enableCnxAutoRead); log.info("Using resource group {} rate limiter for topic {}", rgName, topic); - return; } } else { if (this.resourceGroupRateLimitingEnabled) { - this.resourceGroupPublishLimiter.unregisterRateLimitFunction(this.getName()); this.resourceGroupPublishLimiter = null; this.resourceGroupRateLimitingEnabled = false; } - /* Namespace detached from resource group. Enable the producer read */ - enableProducerReadForPublishRateLimiting(); } + updateActiveRateLimiters(); } public void updateEntryFilters() { @@ -1249,6 +1181,10 @@ public boolean deletePartitionedTopicMetadataWhileInactive() { protected abstract boolean isMigrated(); + public boolean isTransferring() { + return transferring; + } + private static final Logger log = LoggerFactory.getLogger(AbstractTopic.class); public InactiveTopicPolicies getInactiveTopicPolicies() { @@ -1292,37 +1228,15 @@ protected boolean isExceedMaximumMessageSize(int size, PublishContext publishCon * update topic publish dispatcher for this topic. */ public void updatePublishDispatcher() { - synchronized (topicPublishRateLimiterLock) { - PublishRate publishRate = topicPolicies.getPublishRate().get(); - if (publishRate.publishThrottlingRateInByte > 0 || publishRate.publishThrottlingRateInMsg > 0) { - log.info("Enabling publish rate limiting {} on topic {}", publishRate, getName()); - if (!preciseTopicPublishRateLimitingEnable) { - this.brokerService.setupTopicPublishRateLimiterMonitor(); - } - - if (this.topicPublishRateLimiter == null - || this.topicPublishRateLimiter == PublishRateLimiter.DISABLED_RATE_LIMITER) { - // create new rateLimiter if rate-limiter is disabled - if (preciseTopicPublishRateLimitingEnable) { - this.topicPublishRateLimiter = new PrecisePublishLimiter(publishRate, - () -> this.enableCnxAutoRead(), brokerService.pulsar().getExecutor()); - } else { - this.topicPublishRateLimiter = new PublishRateLimiterImpl(publishRate); - } - } else { - this.topicPublishRateLimiter.update(publishRate); - } - } else { - if (log.isDebugEnabled()) { - log.debug("Disabling publish throttling for {}", this.topic); - } - if (topicPublishRateLimiter != null) { - topicPublishRateLimiter.close(); - } - this.topicPublishRateLimiter = PublishRateLimiter.DISABLED_RATE_LIMITER; - enableProducerReadForPublishRateLimiting(); + PublishRate publishRate = topicPolicies.getPublishRate().get(); + if (publishRate.publishThrottlingRateInByte > 0 || publishRate.publishThrottlingRateInMsg > 0) { + log.info("Enabling publish rate limiting {} on topic {}", publishRate, getName()); + } else { + if (log.isDebugEnabled()) { + log.debug("Disabling publish throttling for {}", this.topic); } } + this.topicPublishRateLimiter.update(publishRate); } // subscriptionTypesEnabled is dynamic and can be updated online. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 43bf60f282e09..4077762bb0640 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -26,6 +26,7 @@ import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Queues; +import com.google.common.util.concurrent.RateLimiter; import io.netty.bootstrap.ServerBootstrap; import io.netty.buffer.ByteBuf; import io.netty.channel.AdaptiveRecvByteBufAllocator; @@ -103,7 +104,7 @@ import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl; import org.apache.pulsar.broker.loadbalance.LoadManager; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.DynamicConfigurationResources; import org.apache.pulsar.broker.resources.LocalPoliciesResources; @@ -169,7 +170,6 @@ import org.apache.pulsar.common.util.FieldParser; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.GracefulExecutorServicesShutdown; -import org.apache.pulsar.common.util.RateLimiter; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.common.util.netty.ChannelFutures; @@ -242,10 +242,8 @@ public class BrokerService implements Closeable { private final ScheduledExecutorService messageExpiryMonitor; private final ScheduledExecutorService compactionMonitor; private final ScheduledExecutorService consumedLedgersMonitor; - protected final PublishRateLimiterMonitor topicPublishRateLimiterMonitor; - protected final PublishRateLimiterMonitor brokerPublishRateLimiterMonitor; private ScheduledExecutorService deduplicationSnapshotMonitor; - protected volatile PublishRateLimiter brokerPublishRateLimiter = PublishRateLimiter.DISABLED_RATE_LIMITER; + protected final PublishRateLimiter brokerPublishRateLimiter; protected volatile DispatchRateLimiter brokerDispatchRateLimiter = null; private DistributedIdGenerator producerNameGenerator; @@ -295,6 +293,7 @@ public class BrokerService implements Closeable { public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception { this.pulsar = pulsar; + this.brokerPublishRateLimiter = new PublishRateLimiterImpl(pulsar.getMonotonicSnapshotClock()); this.preciseTopicPublishRateLimitingEnable = pulsar.getConfiguration().isPreciseTopicPublishRateLimiterEnable(); this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); @@ -358,10 +357,6 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws .name("pulsar-consumed-ledgers-monitor") .numThreads(1) .build(); - this.topicPublishRateLimiterMonitor = - new PublishRateLimiterMonitor("pulsar-topic-publish-rate-limiter-monitor"); - this.brokerPublishRateLimiterMonitor = - new PublishRateLimiterMonitor("pulsar-broker-publish-rate-limiter-monitor"); this.backlogQuotaManager = new BacklogQuotaManager(pulsar); this.backlogQuotaChecker = OrderedScheduler.newSchedulerBuilder() .name("pulsar-backlog-quota-checker") @@ -419,7 +414,7 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.brokerEntryPayloadProcessors = BrokerEntryMetadataUtils.loadInterceptors(pulsar.getConfiguration() .getBrokerEntryPayloadProcessors(), BrokerService.class.getClassLoader()); - this.bundlesQuotas = new BundlesQuotas(pulsar.getLocalMetadataStore()); + this.bundlesQuotas = new BundlesQuotas(pulsar); } public void addTopicEventListener(TopicEventsListener... listeners) { @@ -669,87 +664,6 @@ protected void startBacklogQuotaChecker() { } - /** - * Schedules and monitors publish-throttling for all owned topics that has publish-throttling configured. It also - * disables and shutdowns publish-rate-limiter monitor task if broker disables it. - */ - public void setupTopicPublishRateLimiterMonitor() { - // set topic PublishRateLimiterMonitor - long topicTickTimeMs = pulsar().getConfiguration().getTopicPublisherThrottlingTickTimeMillis(); - if (topicTickTimeMs > 0) { - topicPublishRateLimiterMonitor.startOrUpdate(topicTickTimeMs, - this::checkTopicPublishThrottlingRate, this::refreshTopicPublishRate); - } else { - // disable publish-throttling for all topics - topicPublishRateLimiterMonitor.stop(); - } - } - - /** - * Schedules and monitors publish-throttling for broker that has publish-throttling configured. It also - * disables and shutdowns publish-rate-limiter monitor for broker task if broker disables it. - */ - public void setupBrokerPublishRateLimiterMonitor() { - // set broker PublishRateLimiterMonitor - long brokerTickTimeMs = pulsar().getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(); - if (brokerTickTimeMs > 0) { - brokerPublishRateLimiterMonitor.startOrUpdate(brokerTickTimeMs, - this::checkBrokerPublishThrottlingRate, this::refreshBrokerPublishRate); - } else { - // disable publish-throttling for broker. - brokerPublishRateLimiterMonitor.stop(); - } - } - - protected static class PublishRateLimiterMonitor { - private final String name; - private ScheduledExecutorService scheduler = null; - private long tickTimeMs = 0; - private Runnable refreshTask; - - public PublishRateLimiterMonitor(String name) { - this.name = name; - } - - synchronized void startOrUpdate(long tickTimeMs, Runnable checkTask, Runnable refreshTask) { - if (this.scheduler != null) { - // we have old task running. - if (this.tickTimeMs == tickTimeMs) { - // tick time not changed. - return; - } - stop(); - } - //start monitor. - scheduler = OrderedScheduler.newSchedulerBuilder() - .name(name) - .numThreads(1) - .build(); - // schedule task that sums up publish-rate across all cnx on a topic , - // and check the rate limit exceeded or not. - scheduler.scheduleAtFixedRate(checkTask, tickTimeMs, tickTimeMs, TimeUnit.MILLISECONDS); - // schedule task that refreshes rate-limiting bucket - scheduler.scheduleAtFixedRate(refreshTask, 1, 1, TimeUnit.SECONDS); - this.tickTimeMs = tickTimeMs; - this.refreshTask = refreshTask; - } - - synchronized void stop() { - if (this.scheduler != null) { - this.scheduler.shutdownNow(); - // make sure topics are not being throttled - refreshTask.run(); - this.scheduler = null; - this.tickTimeMs = 0; - } - } - - @VisibleForTesting - protected synchronized long getTickTimeMs() { - return tickTimeMs; - } - } - public void close() throws IOException { try { closeAsync().get(); @@ -881,8 +795,6 @@ public CompletableFuture closeAsync() { consumedLedgersMonitor, backlogQuotaChecker, topicOrderedExecutor, - topicPublishRateLimiterMonitor.scheduler, - brokerPublishRateLimiterMonitor.scheduler, deduplicationSnapshotMonitor) .handle()); @@ -972,26 +884,21 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl Set serviceUnits = pulsar.getNamespaceService() != null ? pulsar.getNamespaceService().getOwnedServiceUnits() : null; if (serviceUnits != null) { - try (RateLimiter rateLimiter = maxConcurrentUnload > 0 ? RateLimiter.builder() - .scheduledExecutorService(pulsar.getExecutor()) - .rateTime(1).timeUnit(TimeUnit.SECONDS) - .permits(maxConcurrentUnload).build() : null) { - serviceUnits.forEach(su -> { - if (su != null) { - try { - if (rateLimiter != null) { - rateLimiter.acquire(1); - } - long timeout = pulsar.getConfiguration().getNamespaceBundleUnloadingTimeoutMs(); - pulsar.getNamespaceService().unloadNamespaceBundle(su, timeout, TimeUnit.MILLISECONDS, - closeWithoutWaitingClientDisconnect).get(timeout, TimeUnit.MILLISECONDS); - } catch (Exception e) { - log.warn("Failed to unload namespace bundle {}", su, e); + RateLimiter rateLimiter = maxConcurrentUnload > 0 ? RateLimiter.create(maxConcurrentUnload) : null; + serviceUnits.forEach(su -> { + if (su != null) { + try { + if (rateLimiter != null) { + rateLimiter.acquire(1); } + long timeout = pulsar.getConfiguration().getNamespaceBundleUnloadingTimeoutMs(); + pulsar.getNamespaceService().unloadNamespaceBundle(su, timeout, TimeUnit.MILLISECONDS, + closeWithoutWaitingClientDisconnect).get(timeout, TimeUnit.MILLISECONDS); + } catch (Exception e) { + log.warn("Failed to unload namespace bundle {}", su, e); } - }); - } - + } + }); double closeTopicsTimeSeconds = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - closeTopicsStartTime)) / 1000.0; @@ -2138,26 +2045,6 @@ public void checkInactiveSubscriptions() { forEachTopic(Topic::checkInactiveSubscriptions); } - public void checkTopicPublishThrottlingRate() { - forEachTopic(Topic::checkTopicPublishThrottlingRate); - } - - private void refreshTopicPublishRate() { - forEachTopic(Topic::resetTopicPublishCountAndEnableReadIfRequired); - } - - public void checkBrokerPublishThrottlingRate() { - brokerPublishRateLimiter.checkPublishRate(); - if (brokerPublishRateLimiter.isPublishRateExceeded()) { - forEachTopic(topic -> ((AbstractTopic) topic).disableProducerRead()); - } - } - - private void refreshBrokerPublishRate() { - boolean doneReset = brokerPublishRateLimiter.resetPublishCount(); - forEachTopic(topic -> topic.resetBrokerPublishCountAndEnableReadIfRequired(doneReset)); - } - /** * Iterates over all loaded topics in the broker. */ @@ -2514,22 +2401,25 @@ private void handleDynamicConfigurationUpdates() { return; } Field configField = configFieldWrapper.field; - Object newValue = FieldParser.value(data.get(configKey), configField); - if (configField != null) { - Consumer listener = configRegisteredListeners.get(configKey); - try { - Object existingValue = configField.get(pulsar.getConfiguration()); + Consumer listener = configRegisteredListeners.get(configKey); + try { + final Object existingValue; + final Object newValue; + if (configField != null) { + newValue = FieldParser.value(data.get(configKey), configField); + existingValue = configField.get(pulsar.getConfiguration()); configField.set(pulsar.getConfiguration(), newValue); - log.info("Successfully updated configuration {}/{}", configKey, - data.get(configKey)); - if (listener != null && !existingValue.equals(newValue)) { - listener.accept(newValue); - } - } catch (Exception e) { - log.error("Failed to update config {}/{}", configKey, newValue); + } else { + newValue = value; + existingValue = configFieldWrapper.customValue; + configFieldWrapper.customValue = newValue == null ? null : String.valueOf(newValue); } - } else { - log.error("Found non-dynamic field in dynamicConfigMap {}/{}", configKey, newValue); + log.info("Successfully updated configuration {}/{}", configKey, data.get(configKey)); + if (listener != null && !Objects.equals(existingValue, newValue)) { + listener.accept(newValue); + } + } catch (Exception e) { + log.error("Failed to update config {}", configKey, e); } }); }); @@ -2727,12 +2617,6 @@ private void updateConfigurationAndRegisterListeners() { updateReplicatorMessageDispatchRate(); }); - // add listener to notify broker publish-rate monitoring - registerConfigurationListener("brokerPublisherThrottlingTickTimeMillis", - (publisherThrottlingTickTimeMillis) -> { - setupBrokerPublishRateLimiterMonitor(); - }); - // add listener to update topic publish-rate dynamic config registerConfigurationListener("maxPublishRatePerTopicInMessages", maxPublishRatePerTopicInMessages -> updateMaxPublishRatePerTopicInMessages() @@ -2761,13 +2645,6 @@ private void updateConfigurationAndRegisterListeners() { registerConfigurationListener("dispatchThrottlingRateInByte", (dispatchThrottlingRateInByte) -> updateBrokerDispatchThrottlingMaxRate()); - // add listener to notify topic publish-rate monitoring - if (!preciseTopicPublishRateLimitingEnable) { - registerConfigurationListener("topicPublisherThrottlingTickTimeMillis", - (publisherThrottlingTickTimeMillis) -> { - setupTopicPublishRateLimiterMonitor(); - }); - } // add listener to notify topic subscriptionTypesEnabled changed. registerConfigurationListener("subscriptionTypesEnabled", this::updateBrokerSubscriptionTypesEnabled); @@ -2847,29 +2724,11 @@ private void updateSubscribeRate() { private void updateBrokerPublisherThrottlingMaxRate() { int currentMaxMessageRate = pulsar.getConfiguration().getBrokerPublisherThrottlingMaxMessageRate(); long currentMaxByteRate = pulsar.getConfiguration().getBrokerPublisherThrottlingMaxByteRate(); - int brokerTickMs = pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(); - - // not enable - if (brokerTickMs <= 0 || (currentMaxByteRate <= 0 && currentMaxMessageRate <= 0)) { - if (brokerPublishRateLimiter != PublishRateLimiter.DISABLED_RATE_LIMITER) { - refreshBrokerPublishRate(); - brokerPublishRateLimiter = PublishRateLimiter.DISABLED_RATE_LIMITER; - } - return; - } final PublishRate publishRate = new PublishRate(currentMaxMessageRate, currentMaxByteRate); log.info("Update broker publish rate limiting {}", publishRate); - // lazy init broker Publish-rateLimiting monitoring if not initialized yet - this.setupBrokerPublishRateLimiterMonitor(); - if (brokerPublishRateLimiter == null - || brokerPublishRateLimiter == PublishRateLimiter.DISABLED_RATE_LIMITER) { - // create new rateLimiter if rate-limiter is disabled - brokerPublishRateLimiter = new PublishRateLimiterImpl(publishRate); - } else { - brokerPublishRateLimiter.update(publishRate); - } + brokerPublishRateLimiter.update(publishRate); } private void updateTopicMessageDispatchRate() { @@ -2968,18 +2827,25 @@ public void registerConfigurationListener(String configKey, Consumer list private void addDynamicConfigValidator(String key, Predicate validator) { validateConfigKey(key); - if (dynamicConfigurationMap.containsKey(key)) { - dynamicConfigurationMap.get(key).validator = validator; - } + dynamicConfigurationMap.get(key).validator = validator; } private void validateConfigKey(String key) { - try { - ServiceConfiguration.class.getDeclaredField(key); - } catch (Exception e) { - log.error("ServiceConfiguration key {} not found {}", key, e.getMessage()); - throw new IllegalArgumentException("Invalid service config " + key, e); + if (!dynamicConfigurationMap.containsKey(key)) { + throw new IllegalArgumentException(key + " doesn't exits in the dynamicConfigurationMap"); + } + } + + /** + * Allows the third-party plugin to register a custom dynamic configuration. + */ + public void registerCustomDynamicConfiguration(String key, Predicate validator) { + if (dynamicConfigurationMap.containsKey(key)) { + throw new IllegalArgumentException(key + " already exists in the dynamicConfigurationMap"); } + ConfigField configField = ConfigField.newCustomConfigField(null); + configField.validator = validator; + dynamicConfigurationMap.put(key, configField); } private void createDynamicConfigPathIfNotExist() { @@ -3356,13 +3222,24 @@ public void unblockDispatchersOnUnAckMessages(List validator; public ConfigField(Field field) { super(); this.field = field; } + + public static ConfigField newCustomConfigField(String customValue) { + ConfigField configField = new ConfigField(null); + configField.customValue = customValue; + return configField; + } } /** @@ -3411,10 +3288,10 @@ private CompletableFuture isAllowAutoTopicCreationAsync(final TopicName return CompletableFuture.completedFuture(false); } - // ServiceUnitStateChannelImpl.TOPIC expects to be a non-partitioned-topic now. + // ExtensibleLoadManagerImpl.internal topics expects to be non-partitioned-topics now. // We don't allow the auto-creation here. - // ServiceUnitStateChannelImpl.start() is responsible to create the topic. - if (ServiceUnitStateChannelImpl.TOPIC.equals(topicName.toString())) { + // ExtensibleLoadManagerImpl.start() is responsible to create the internal system topics. + if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { return CompletableFuture.completedFuture(false); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index ee4fcff3ad1aa..83dcd8d6c1616 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -44,6 +44,7 @@ import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageId; @@ -407,8 +408,12 @@ public void disconnect() { } public void disconnect(boolean isResetCursor) { + disconnect(isResetCursor, Optional.empty()); + } + + public void disconnect(boolean isResetCursor, Optional assignedBrokerLookupData) { log.info("Disconnecting consumer: {}", this); - cnx.closeConsumer(this); + cnx.closeConsumer(this, assignedBrokerLookupData); try { close(isResetCursor); } catch (BrokerServiceException e) { @@ -416,8 +421,8 @@ public void disconnect(boolean isResetCursor) { } } - public void doUnsubscribe(final long requestId) { - subscription.doUnsubscribe(this).thenAccept(v -> { + public void doUnsubscribe(final long requestId, boolean force) { + subscription.doUnsubscribe(this, force).thenAccept(v -> { log.info("Unsubscribed successfully from {}", subscription); cnx.removedConsumer(this); cnx.getCommandSender().sendSuccessResponse(requestId); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index 3ca06dc83d9aa..08e5caaa2ddd5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -16,12 +16,14 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.pulsar.broker.service; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.MessageMetadata; @@ -49,7 +51,11 @@ public interface Dispatcher { * * @return */ - CompletableFuture close(); + default CompletableFuture close() { + return close(true, Optional.empty()); + } + + CompletableFuture close(boolean disconnectClients, Optional assignedBrokerLookupData); boolean isClosed(); @@ -63,12 +69,17 @@ public interface Dispatcher { * * @return */ - CompletableFuture disconnectAllConsumers(boolean isResetCursor); + default CompletableFuture disconnectAllConsumers(boolean isResetCursor) { + return disconnectAllConsumers(isResetCursor, Optional.empty()); + } default CompletableFuture disconnectAllConsumers() { return disconnectAllConsumers(false); } + CompletableFuture disconnectAllConsumers(boolean isResetCursor, + Optional assignedBrokerLookupData); + void resetCloseFuture(); /** @@ -91,7 +102,8 @@ default Optional getRateLimiter() { } default void updateRateLimiter() { - //No-op + initializeDispatchRateLimiterIfNeeded(); + getRateLimiter().ifPresent(DispatchRateLimiter::updateDispatchRate); } default boolean initializeDispatchRateLimiterIfNeeded() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PrecisePublishLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PrecisePublishLimiter.java deleted file mode 100644 index ce14f6d7dd71e..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PrecisePublishLimiter.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.service; - -import java.util.concurrent.ScheduledExecutorService; -import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.PublishRate; -import org.apache.pulsar.common.util.RateLimitFunction; -import org.apache.pulsar.common.util.RateLimiter; - -public class PrecisePublishLimiter implements PublishRateLimiter { - protected volatile int publishMaxMessageRate = 0; - protected volatile long publishMaxByteRate = 0; - // precise mode for publish rate limiter - private volatile RateLimiter topicPublishRateLimiterOnMessage; - private volatile RateLimiter topicPublishRateLimiterOnByte; - private final RateLimitFunction rateLimitFunction; - private final ScheduledExecutorService scheduledExecutorService; - - public PrecisePublishLimiter(Policies policies, String clusterName, RateLimitFunction rateLimitFunction) { - this.rateLimitFunction = rateLimitFunction; - update(policies, clusterName); - this.scheduledExecutorService = null; - } - - public PrecisePublishLimiter(PublishRate publishRate, RateLimitFunction rateLimitFunction) { - this(publishRate, rateLimitFunction, null); - } - - public PrecisePublishLimiter(PublishRate publishRate, RateLimitFunction rateLimitFunction, - ScheduledExecutorService scheduledExecutorService) { - this.rateLimitFunction = rateLimitFunction; - update(publishRate); - this.scheduledExecutorService = scheduledExecutorService; - } - - @Override - public void checkPublishRate() { - // No-op - } - - @Override - public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { - // No-op - } - - @Override - public boolean resetPublishCount() { - return true; - } - - @Override - public boolean isPublishRateExceeded() { - return false; - } - - // If all rate limiters are not exceeded, re-enable auto read from socket. - private void tryReleaseConnectionThrottle() { - RateLimiter currentTopicPublishRateLimiterOnMessage = topicPublishRateLimiterOnMessage; - RateLimiter currentTopicPublishRateLimiterOnByte = topicPublishRateLimiterOnByte; - if ((currentTopicPublishRateLimiterOnMessage != null - && currentTopicPublishRateLimiterOnMessage.getAvailablePermits() <= 0) - || (currentTopicPublishRateLimiterOnByte != null - && currentTopicPublishRateLimiterOnByte.getAvailablePermits() <= 0)) { - return; - } - this.rateLimitFunction.apply(); - } - - @Override - public void update(Policies policies, String clusterName) { - final PublishRate maxPublishRate = policies.publishMaxMessageRate != null - ? policies.publishMaxMessageRate.get(clusterName) - : null; - this.update(maxPublishRate); - } - - public void update(PublishRate maxPublishRate) { - replaceLimiters(() -> { - if (maxPublishRate != null - && (maxPublishRate.publishThrottlingRateInMsg > 0 - || maxPublishRate.publishThrottlingRateInByte > 0)) { - this.publishMaxMessageRate = Math.max(maxPublishRate.publishThrottlingRateInMsg, 0); - this.publishMaxByteRate = Math.max(maxPublishRate.publishThrottlingRateInByte, 0); - if (this.publishMaxMessageRate > 0) { - topicPublishRateLimiterOnMessage = - RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(publishMaxMessageRate) - .rateLimitFunction(this::tryReleaseConnectionThrottle) - .isDispatchOrPrecisePublishRateLimiter(true) - .build(); - } - if (this.publishMaxByteRate > 0) { - topicPublishRateLimiterOnByte = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(publishMaxByteRate) - .rateLimitFunction(this::tryReleaseConnectionThrottle) - .isDispatchOrPrecisePublishRateLimiter(true) - .build(); - } - } else { - this.publishMaxMessageRate = 0; - this.publishMaxByteRate = 0; - } - }); - } - - @Override - public boolean tryAcquire(int numbers, long bytes) { - RateLimiter currentTopicPublishRateLimiterOnMessage = topicPublishRateLimiterOnMessage; - RateLimiter currentTopicPublishRateLimiterOnByte = topicPublishRateLimiterOnByte; - return (currentTopicPublishRateLimiterOnMessage == null - || currentTopicPublishRateLimiterOnMessage.tryAcquire(numbers)) - && (currentTopicPublishRateLimiterOnByte == null - || currentTopicPublishRateLimiterOnByte.tryAcquire(bytes)); - } - - @Override - public void close() { - rateLimitFunction.apply(); - replaceLimiters(null); - } - - private void replaceLimiters(Runnable updater) { - RateLimiter previousTopicPublishRateLimiterOnMessage = topicPublishRateLimiterOnMessage; - topicPublishRateLimiterOnMessage = null; - RateLimiter previousTopicPublishRateLimiterOnByte = topicPublishRateLimiterOnByte; - topicPublishRateLimiterOnByte = null; - try { - if (updater != null) { - updater.run(); - } - } finally { - // Close previous limiters to prevent resource leakages. - // Delay closing of previous limiters after new ones are in place so that updating the limiter - // doesn't cause unavailability. - if (previousTopicPublishRateLimiterOnMessage != null) { - previousTopicPublishRateLimiterOnMessage.close(); - } - if (previousTopicPublishRateLimiterOnByte != null) { - previousTopicPublishRateLimiterOnByte.close(); - } - } - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index 1a4490d6b1b46..7e4459505a523 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -321,7 +321,7 @@ private void startPublishOperation(int batchSize, long msgSize) { // barrier pendingPublishAcksUpdater.lazySet(this, pendingPublishAcks + 1); // increment publish-count - this.getTopic().incrementPublishCount(batchSize, msgSize); + this.getTopic().incrementPublishCount(this, batchSize, msgSize); } private void publishOperationCompleted() { @@ -488,9 +488,16 @@ public void completed(Exception exception, long ledgerId, long entryId) { final ServerError serverError = getServerError(exception); producer.cnx.execute(() -> { - if (!(exception instanceof TopicClosedException)) { + // if the topic is transferring, we don't send error code to the clients. + if (producer.getTopic().isTransferring()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Received producer exception: {} while transferring.", + producer.getTopic().getName(), exception.getMessage(), exception); + } + } else if (!(exception instanceof TopicClosedException)) { // For TopicClosed exception there's no need to send explicit error, since the client was // already notified + // For TopicClosingOrDeleting exception, a notification will be sent separately long callBackSequenceId = Math.max(highestSequenceId, sequenceId); producer.cnx.getCommandSender().sendSendError(producer.producerId, callBackSequenceId, serverError, exception.getMessage()); @@ -712,7 +719,7 @@ public CompletableFuture disconnect() { */ public CompletableFuture disconnect(Optional assignedBrokerLookupData) { if (!closeFuture.isDone() && isDisconnecting.compareAndSet(false, true)) { - log.info("Disconnecting producer: {}", this); + log.info("Disconnecting producer: {}, assignedBrokerLookupData: {}", this, assignedBrokerLookupData); cnx.execute(() -> { cnx.closeProducer(this, assignedBrokerLookupData); closeNow(true); @@ -846,4 +853,27 @@ public boolean isDisconnecting() { private static final Logger log = LoggerFactory.getLogger(Producer.class); + /** + * This method increments a counter that is used to control the throttling of a connection. + * The connection's read operations are paused when the counter's value is greater than 0, indicating that + * throttling is in effect. + * It's important to note that after calling this method, it is the caller's responsibility to ensure that the + * counter is decremented by calling the {@link #decrementThrottleCount()} method when throttling is no longer + * needed on the connection. + */ + public void incrementThrottleCount() { + cnx.incrementThrottleCount(); + } + + /** + * This method decrements a counter that is used to control the throttling of a connection. + * The connection's read operations are resumed when the counter's value is 0, indicating that + * throttling is no longer in effect. + * It's important to note that before calling this method, the caller should have previously + * incremented the counter by calling the {@link #incrementThrottleCount()} method when throttling + * was needed on the connection. + */ + public void decrementThrottleCount() { + cnx.decrementThrottleCount(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiter.java index fde2e7cb56dce..cdc2f448abd32 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiter.java @@ -21,35 +21,19 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PublishRate; -public interface PublishRateLimiter extends AutoCloseable { - - PublishRateLimiter DISABLED_RATE_LIMITER = PublishRateLimiterDisable.DISABLED_RATE_LIMITER; +public interface PublishRateLimiter { /** - * checks and update state of current publish and marks if it has exceeded the rate-limiting threshold. - */ - void checkPublishRate(); - - /** - * increments current publish count. + * Consumes publishing quota and handles throttling. + *

+ * The rate limiter implementation calls {@link Producer#incrementThrottleCount()} to indicate + * that the producer should be throttled. The rate limiter must schedule a call to + * {@link Producer#decrementThrottleCount()} after a throttling period that it calculates. * - * @param numOfMessages - * @param msgSizeInBytes - */ - void incrementPublishCount(int numOfMessages, long msgSizeInBytes); - - /** - * reset current publish count. - * - * @return - */ - boolean resetPublishCount(); - - /** - * returns true if current publish has reached the rate-limiting threshold. - * @return + * @param numOfMessages number of messages to publish + * @param msgSizeInBytes size of messages in bytes to publish */ - boolean isPublishRateExceeded(); + void handlePublishThrottling(Producer producer, int numOfMessages, long msgSizeInBytes); /** * updates rate-limiting threshold based on policies. @@ -63,17 +47,4 @@ public interface PublishRateLimiter extends AutoCloseable { * @param maxPublishRate */ void update(PublishRate maxPublishRate); - - /** - * try to acquire permit. - * - * @param numbers - * @param bytes - */ - boolean tryAcquire(int numbers, long bytes); - - /** - * Close the limiter. - */ - void close(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterDisable.java deleted file mode 100644 index fdc13ed8f4e31..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterDisable.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.service; - -import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.PublishRate; - -public class PublishRateLimiterDisable implements PublishRateLimiter { - - public static final PublishRateLimiterDisable DISABLED_RATE_LIMITER = new PublishRateLimiterDisable(); - - @Override - public void checkPublishRate() { - // No-op - } - - @Override - public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { - // No-op - } - - @Override - public boolean resetPublishCount() { - // No-op - return false; - } - - @Override - public boolean isPublishRateExceeded() { - return false; - } - - @Override - public void update(Policies policies, String clusterName) { - // No-op - } - - @Override - public void update(PublishRate maxPublishRate) { - // No-op - } - - @Override - public boolean tryAcquire(int numbers, long bytes) { - // Always allow - return true; - } - - @Override - public void close() { - // No-op - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java index 1947acd87fc66..8255d9b6931ff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java @@ -16,70 +16,135 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.pulsar.broker.service; -import java.util.concurrent.atomic.LongAdder; +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.EventLoopGroup; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PublishRate; +import org.jctools.queues.MessagePassingQueue; +import org.jctools.queues.MpscUnboundedArrayQueue; public class PublishRateLimiterImpl implements PublishRateLimiter { - protected volatile int publishMaxMessageRate = 0; - protected volatile long publishMaxByteRate = 0; - protected volatile boolean publishThrottlingEnabled = false; - protected volatile boolean publishRateExceeded = false; - protected volatile LongAdder currentPublishMsgCount = new LongAdder(); - protected volatile LongAdder currentPublishByteCount = new LongAdder(); - - public PublishRateLimiterImpl(Policies policies, String clusterName) { - update(policies, clusterName); - } + private volatile AsyncTokenBucket tokenBucketOnMessage; + private volatile AsyncTokenBucket tokenBucketOnByte; + private final MonotonicSnapshotClock monotonicSnapshotClock; - public PublishRateLimiterImpl(PublishRate maxPublishRate) { - update(maxPublishRate); + private final MessagePassingQueue unthrottlingQueue = new MpscUnboundedArrayQueue<>(1024); + + private final AtomicInteger throttledProducersCount = new AtomicInteger(0); + private final AtomicBoolean processingQueuedProducers = new AtomicBoolean(false); + + public PublishRateLimiterImpl(MonotonicSnapshotClock monotonicSnapshotClock) { + this.monotonicSnapshotClock = monotonicSnapshotClock; } + /** + * {@inheritDoc} + */ @Override - public void checkPublishRate() { - if (this.publishThrottlingEnabled && !publishRateExceeded) { - if (this.publishMaxByteRate > 0) { - long currentPublishByteRate = this.currentPublishByteCount.sum(); - if (currentPublishByteRate > this.publishMaxByteRate) { - publishRateExceeded = true; - return; - } - } - - if (this.publishMaxMessageRate > 0) { - long currentPublishMsgRate = this.currentPublishMsgCount.sum(); - if (currentPublishMsgRate > this.publishMaxMessageRate) { - publishRateExceeded = true; - } - } + public void handlePublishThrottling(Producer producer, int numOfMessages, + long msgSizeInBytes) { + boolean shouldThrottle = false; + AsyncTokenBucket currentTokenBucketOnMessage = tokenBucketOnMessage; + if (currentTokenBucketOnMessage != null) { + // consume tokens from the token bucket for messages + // we should throttle if it returns false since the token bucket is empty in that case + shouldThrottle = !currentTokenBucketOnMessage.consumeTokensAndCheckIfContainsTokens(numOfMessages); + } + AsyncTokenBucket currentTokenBucketOnByte = tokenBucketOnByte; + if (currentTokenBucketOnByte != null) { + // consume tokens from the token bucket for bytes + // we should throttle if it returns false since the token bucket is empty in that case + shouldThrottle |= !currentTokenBucketOnByte.consumeTokensAndCheckIfContainsTokens(msgSizeInBytes); + } + if (shouldThrottle) { + // throttle the producer by incrementing the throttle count + producer.incrementThrottleCount(); + // schedule decrementing the throttle count to possibly unthrottle the producer after the + // throttling period + scheduleDecrementThrottleCount(producer); } } - @Override - public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { - if (this.publishThrottlingEnabled) { - this.currentPublishMsgCount.add(numOfMessages); - this.currentPublishByteCount.add(msgSizeInBytes); + private void scheduleDecrementThrottleCount(Producer producer) { + // add the producer to the queue of producers to be unthrottled + unthrottlingQueue.offer(producer); + // schedule unthrottling when the throttling count is incremented to 1 + // this is to avoid scheduling unthrottling multiple times for concurrent producers + if (throttledProducersCount.incrementAndGet() == 1) { + EventLoopGroup executor = producer.getCnx().getBrokerService().executor(); + scheduleUnthrottling(executor, calculateThrottlingDurationNanos()); } } - @Override - public boolean resetPublishCount() { - if (this.publishThrottlingEnabled) { - this.currentPublishMsgCount.reset(); - this.currentPublishByteCount.reset(); - this.publishRateExceeded = false; - return true; + /** + * Schedules the unthrottling operation after a throttling period. + * + * This method will usually be called only once at a time. However, in a multi-threaded environment, + * it's possible for concurrent threads to call this method simultaneously. This is acceptable and does not + * disrupt the functionality, as the method is designed to handle such scenarios gracefully. + * + * The solution avoids using locks and this nonblocking approach requires allowing concurrent calls to this method. + * The implementation intends to prevent skipping of scheduling as a result of a race condition, which could + * result in a producer never being unthrottled. + * + * The solution for skipping of scheduling is to allow 2 threads to schedule unthrottling when the throttling + * count is exactly 1 when unthrottleQueuedProducers checks whether there's a need to reschedule. There might + * be another thread that added it and also scheduled unthrottling. This is acceptable and intended for resolving + * the race condition. + * + * @param executor The executor service used to schedule the unthrottling operation. + * @param delayNanos + */ + private void scheduleUnthrottling(ScheduledExecutorService executor, long delayNanos) { + executor.schedule(() -> this.unthrottleQueuedProducers(executor), delayNanos, + TimeUnit.NANOSECONDS); + } + + private long calculateThrottlingDurationNanos() { + AsyncTokenBucket currentTokenBucketOnMessage = tokenBucketOnMessage; + long throttlingDurationNanos = 0L; + if (currentTokenBucketOnMessage != null) { + throttlingDurationNanos = currentTokenBucketOnMessage.calculateThrottlingDuration(); } - return false; + AsyncTokenBucket currentTokenBucketOnByte = tokenBucketOnByte; + if (currentTokenBucketOnByte != null) { + throttlingDurationNanos = Math.max(throttlingDurationNanos, + currentTokenBucketOnByte.calculateThrottlingDuration()); + } + return throttlingDurationNanos; } - @Override - public boolean isPublishRateExceeded() { - return publishRateExceeded; + private void unthrottleQueuedProducers(ScheduledExecutorService executor) { + if (!processingQueuedProducers.compareAndSet(false, true)) { + // another thread is already processing unthrottling + return; + } + try { + Producer producer; + long throttlingDuration = 0L; + // unthrottle as many producers as possible while there are token available + while ((throttlingDuration = calculateThrottlingDurationNanos()) == 0L + && (producer = unthrottlingQueue.poll()) != null) { + producer.decrementThrottleCount(); + throttledProducersCount.decrementAndGet(); + } + // if there are still producers to be unthrottled, schedule unthrottling again + // after another throttling period + if (throttledProducersCount.get() > 0) { + scheduleUnthrottling(executor, throttlingDuration); + } + } finally { + processingQueuedProducers.set(false); + } } @Override @@ -91,26 +156,36 @@ public void update(Policies policies, String clusterName) { } public void update(PublishRate maxPublishRate) { - if (maxPublishRate != null - && (maxPublishRate.publishThrottlingRateInMsg > 0 || maxPublishRate.publishThrottlingRateInByte > 0)) { - this.publishThrottlingEnabled = true; - this.publishMaxMessageRate = Math.max(maxPublishRate.publishThrottlingRateInMsg, 0); - this.publishMaxByteRate = Math.max(maxPublishRate.publishThrottlingRateInByte, 0); + if (maxPublishRate != null) { + updateTokenBuckets(maxPublishRate.publishThrottlingRateInMsg, maxPublishRate.publishThrottlingRateInByte); } else { - this.publishMaxMessageRate = 0; - this.publishMaxByteRate = 0; - this.publishThrottlingEnabled = false; + tokenBucketOnMessage = null; + tokenBucketOnByte = null; } - resetPublishCount(); } - @Override - public boolean tryAcquire(int numbers, long bytes) { - return false; + protected void updateTokenBuckets(long publishThrottlingRateInMsg, long publishThrottlingRateInByte) { + if (publishThrottlingRateInMsg > 0) { + tokenBucketOnMessage = + AsyncTokenBucket.builder().rate(publishThrottlingRateInMsg).clock(monotonicSnapshotClock).build(); + } else { + tokenBucketOnMessage = null; + } + if (publishThrottlingRateInByte > 0) { + tokenBucketOnByte = + AsyncTokenBucket.builder().rate(publishThrottlingRateInByte).clock(monotonicSnapshotClock).build(); + } else { + tokenBucketOnByte = null; + } } - @Override - public void close() { - // no-op + @VisibleForTesting + public AsyncTokenBucket getTokenBucketOnMessage() { + return tokenBucketOnMessage; + } + + @VisibleForTesting + public AsyncTokenBucket getTokenBucketOnByte() { + return tokenBucketOnByte; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index debfad3b615b1..9f2b98aeb40d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -28,6 +28,7 @@ import static org.apache.pulsar.broker.service.persistent.PersistentTopic.getMigratedClusterUrl; import static org.apache.pulsar.common.api.proto.ProtocolVersion.v5; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import static org.apache.pulsar.common.protocol.Commands.newCloseConsumer; import static org.apache.pulsar.common.protocol.Commands.newLookupErrorResponse; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; @@ -40,7 +41,6 @@ import io.netty.util.concurrent.FastThreadLocal; import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.ScheduledFuture; -import io.prometheus.client.Gauge; import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; @@ -70,8 +70,6 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.commons.lang3.mutable.MutableInt; -import org.apache.commons.lang3.mutable.MutableLong; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.TransactionMetadataStoreService; @@ -227,11 +225,8 @@ public class ServerCnx extends PulsarHandler implements TransportCnx { private final int maxMessageSize; private boolean preciseDispatcherFlowControl; - private boolean preciseTopicPublishRateLimitingEnable; private boolean encryptionRequireOnProducer; - // Flag to manage throttling-rate by atomically enable/disable read-channel. - private volatile boolean autoReadDisabledRateLimiting = false; private FeatureFlags features; private PulsarCommandSender commandSender; @@ -240,23 +235,52 @@ public class ServerCnx extends PulsarHandler implements TransportCnx { private static final KeySharedMeta emptyKeySharedMeta = new KeySharedMeta() .setKeySharedMode(KeySharedMode.AUTO_SPLIT); - // Flag to manage throttling-publish-buffer by atomically enable/disable read-channel. - private boolean autoReadDisabledPublishBufferLimiting = false; private final long maxPendingBytesPerThread; private final long resumeThresholdPendingBytesPerThread; private final long connectionLivenessCheckTimeoutMillis; - // Number of bytes pending to be published from a single specific IO thread. - private static final FastThreadLocal pendingBytesPerThread = new FastThreadLocal() { - @Override - protected MutableLong initialValue() throws Exception { - return new MutableLong(); + // Tracks and limits number of bytes pending to be published from a single specific IO thread. + static final class PendingBytesPerThreadTracker { + private static final FastThreadLocal pendingBytesPerThread = + new FastThreadLocal<>() { + @Override + protected PendingBytesPerThreadTracker initialValue() throws Exception { + return new PendingBytesPerThreadTracker(); + } + }; + + private long pendingBytes; + private boolean limitExceeded; + + public static PendingBytesPerThreadTracker getInstance() { + return pendingBytesPerThread.get(); } - }; + + public void incrementPublishBytes(long bytes, long maxPendingBytesPerThread) { + pendingBytes += bytes; + // when the limit is exceeded we throttle all connections that are sharing the same thread + if (maxPendingBytesPerThread > 0 && pendingBytes > maxPendingBytesPerThread + && !limitExceeded) { + limitExceeded = true; + cnxsPerThread.get().forEach(cnx -> cnx.throttleTracker.setPublishBufferLimiting(true)); + } + } + + public void decrementPublishBytes(long bytes, long resumeThresholdPendingBytesPerThread) { + pendingBytes -= bytes; + // when the limit has been exceeded, and we are below the resume threshold + // we resume all connections sharing the same thread + if (limitExceeded && pendingBytes <= resumeThresholdPendingBytesPerThread) { + limitExceeded = false; + cnxsPerThread.get().forEach(cnx -> cnx.throttleTracker.setPublishBufferLimiting(false)); + } + } + } + // A set of connections tied to the current thread - private static final FastThreadLocal> cnxsPerThread = new FastThreadLocal>() { + private static final FastThreadLocal> cnxsPerThread = new FastThreadLocal<>() { @Override protected Set initialValue() throws Exception { return Collections.newSetFromMap(new IdentityHashMap<>()); @@ -267,6 +291,9 @@ enum State { Start, Connected, Failed, Connecting } + private final ServerCnxThrottleTracker throttleTracker = new ServerCnxThrottleTracker(this); + + public ServerCnx(PulsarService pulsar) { this(pulsar, null); } @@ -301,7 +328,6 @@ public ServerCnx(PulsarService pulsar, String listenerName) { this.maxPendingSendRequests = conf.getMaxPendingPublishRequestsPerConnection(); this.resumeReadsThreshold = maxPendingSendRequests / 2; this.preciseDispatcherFlowControl = conf.isPreciseDispatcherFlowControl(); - this.preciseTopicPublishRateLimitingEnable = conf.isPreciseTopicPublishRateLimiterEnable(); this.encryptionRequireOnProducer = conf.isEncryptionRequireOnProducer(); // Assign a portion of max-pending bytes to each IO thread this.maxPendingBytesPerThread = conf.getMaxMessagePublishBufferSizeInMB() * 1024L * 1024L @@ -1321,7 +1347,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { topicName, remoteAddress, consumerId); } consumers.remove(consumerId, consumerFuture); - closeConsumer(consumerId); + closeConsumer(consumerId, Optional.empty()); return null; } } else if (exception.getCause() instanceof BrokerServiceException) { @@ -1763,12 +1789,12 @@ protected void handleSend(CommandSend send, ByteBuf headersAndPayload) { } PulsarService pulsar = getBrokerService().pulsar(); - if (producer.getTopic().isFenced() - && ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { + // if the topic is transferring, we ignore send msg. + if (producer.getTopic().isTransferring()) { long ignoredMsgCount = ExtensibleLoadManagerImpl.get(pulsar) - .getIgnoredSendMsgCounter().incrementAndGet(); + .getIgnoredSendMsgCounter().addAndGet(send.getNumMessages()); if (log.isDebugEnabled()) { - log.debug("Ignored send msg from:{}:{} to fenced topic:{} during unloading." + log.debug("Ignored send msg from:{}:{} to fenced topic:{} while transferring." + " Ignored message count:{}.", remoteAddress, send.getProducerId(), producer.getTopic().getName(), ignoredMsgCount); } @@ -1791,7 +1817,7 @@ protected void handleSend(CommandSend send, ByteBuf headersAndPayload) { } } - startSendOperation(producer, headersAndPayload.readableBytes(), send.getNumMessages()); + increasePendingSendRequestsAndPublishBytes(headersAndPayload.readableBytes()); if (send.hasTxnidMostBits() && send.hasTxnidLeastBits()) { TxnID txnID = new TxnID(send.getTxnidMostBits(), send.getTxnidLeastBits()); @@ -1842,6 +1868,15 @@ protected void handleAck(CommandAck ack) { if (consumerFuture != null && consumerFuture.isDone() && !consumerFuture.isCompletedExceptionally()) { Consumer consumer = consumerFuture.getNow(null); + Subscription subscription = consumer.getSubscription(); + if (subscription.getTopic().isTransferring()) { + // Message acks are silently ignored during topic transfer. + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Ignoring message acknowledgment during topic transfer, ack count: {}", + subscription, consumerId, ack.getMessageIdsCount()); + } + return; + } consumer.messageAcked(ack).thenRun(() -> { if (hasRequestId) { writeAndFlush(Commands.newAckResponse( @@ -1923,7 +1958,7 @@ protected void handleUnsubscribe(CommandUnsubscribe unsubscribe) { CompletableFuture consumerFuture = consumers.get(unsubscribe.getConsumerId()); if (consumerFuture != null && consumerFuture.isDone() && !consumerFuture.isCompletedExceptionally()) { - consumerFuture.getNow(null).doUnsubscribe(unsubscribe.getRequestId()); + consumerFuture.getNow(null).doUnsubscribe(unsubscribe.getRequestId(), unsubscribe.isForce()); } else { commandSender.sendErrorResponse(unsubscribe.getRequestId(), ServerError.MetadataError, "Consumer not found"); @@ -2119,23 +2154,28 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) long requestId = getLastMessageId.getRequestId(); Topic topic = consumer.getSubscription().getTopic(); - Position lastPosition = topic.getLastPosition(); - int partitionIndex = TopicName.getPartitionIndex(topic.getName()); - - Position markDeletePosition = null; - if (consumer.getSubscription() instanceof PersistentSubscription) { - markDeletePosition = ((PersistentSubscription) consumer.getSubscription()).getCursor() - .getMarkDeletedPosition(); - } - - getLargestBatchIndexWhenPossible( - topic, - (PositionImpl) lastPosition, - (PositionImpl) markDeletePosition, - partitionIndex, - requestId, - consumer.getSubscription().getName()); + topic.checkIfTransactionBufferRecoverCompletely(true).thenRun(() -> { + Position lastPosition = ((PersistentTopic) topic).getMaxReadPosition(); + int partitionIndex = TopicName.getPartitionIndex(topic.getName()); + + Position markDeletePosition = null; + if (consumer.getSubscription() instanceof PersistentSubscription) { + markDeletePosition = ((PersistentSubscription) consumer.getSubscription()).getCursor() + .getMarkDeletedPosition(); + } + getLargestBatchIndexWhenPossible( + topic, + (PositionImpl) lastPosition, + (PositionImpl) markDeletePosition, + partitionIndex, + requestId, + consumer.getSubscription().getName()); + }).exceptionally(e -> { + writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), + ServerError.UnknownError, "Failed to recover Transaction Buffer.")); + return null; + }); } else { writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), ServerError.MetadataError, "Consumer not found")); @@ -3071,15 +3111,17 @@ private void closeProducer(long producerId, long epoch, Optional assignedBrokerLookupData) { // removes consumer-connection from map and send close command to consumer safelyRemoveConsumer(consumer); - closeConsumer(consumer.consumerId()); + closeConsumer(consumer.consumerId(), assignedBrokerLookupData); } - private void closeConsumer(long consumerId) { + private void closeConsumer(long consumerId, Optional assignedBrokerLookupData) { if (getRemoteEndpointProtocolVersion() >= v5.getValue()) { - writeAndFlush(Commands.newCloseConsumer(consumerId, -1L)); + writeAndFlush(newCloseConsumer(consumerId, -1L, + assignedBrokerLookupData.map(BrokerLookupData::pulsarServiceUrl).orElse(null), + assignedBrokerLookupData.map(BrokerLookupData::pulsarServiceUrlTls).orElse(null))); } else { close(); } @@ -3150,64 +3192,20 @@ public boolean isWritable() { return ctx.channel().isWritable(); } - private static final Gauge throttledConnections = Gauge.build() - .name("pulsar_broker_throttled_connections") - .help("Counter of connections throttled because of per-connection limit") - .register(); - - private static final Gauge throttledConnectionsGlobal = Gauge.build() - .name("pulsar_broker_throttled_connections_global_limit") - .help("Counter of connections throttled because of per-connection limit") - .register(); - - public void startSendOperation(Producer producer, int msgSize, int numMessages) { - boolean isPublishRateExceeded = false; - if (preciseTopicPublishRateLimitingEnable) { - boolean isPreciseTopicPublishRateExceeded = - producer.getTopic().isTopicPublishRateExceeded(numMessages, msgSize); - if (isPreciseTopicPublishRateExceeded) { - producer.getTopic().disableCnxAutoRead(); - return; - } - isPublishRateExceeded = producer.getTopic().isBrokerPublishRateExceeded(); - } else { - if (producer.getTopic().isResourceGroupRateLimitingEnabled()) { - final boolean resourceGroupPublishRateExceeded = - producer.getTopic().isResourceGroupPublishRateExceeded(numMessages, msgSize); - if (resourceGroupPublishRateExceeded) { - producer.getTopic().disableCnxAutoRead(); - return; - } - } - isPublishRateExceeded = producer.getTopic().isPublishRateExceeded(); - } - - if (++pendingSendRequest == maxPendingSendRequests || isPublishRateExceeded) { - // When the quota of pending send requests is reached, stop reading from socket to cause backpressure on - // client connection, possibly shared between multiple producers - disableCnxAutoRead(); - autoReadDisabledRateLimiting = isPublishRateExceeded; - throttledConnections.inc(); - } - - if (pendingBytesPerThread.get().addAndGet(msgSize) >= maxPendingBytesPerThread - && !autoReadDisabledPublishBufferLimiting - && maxPendingBytesPerThread > 0) { - // Disable reading from all the connections associated with this thread - MutableInt pausedConnections = new MutableInt(); - cnxsPerThread.get().forEach(cnx -> { - if (cnx.hasProducers() && !cnx.autoReadDisabledPublishBufferLimiting) { - cnx.disableCnxAutoRead(); - cnx.autoReadDisabledPublishBufferLimiting = true; - pausedConnections.increment(); - } - }); - - getBrokerService().pausedConnections(pausedConnections.intValue()); + // handle throttling based on pending send requests in the same connection + // or the pending publish bytes + private void increasePendingSendRequestsAndPublishBytes(int msgSize) { + if (++pendingSendRequest == maxPendingSendRequests) { + throttleTracker.setPendingSendRequestsExceeded(true); } + PendingBytesPerThreadTracker.getInstance().incrementPublishBytes(msgSize, maxPendingBytesPerThread); } - private void recordRateLimitMetrics(ConcurrentLongHashMap> producers) { + + /** + * Increase the throttling metric for the topic when a producer is throttled. + */ + void increasePublishLimitedTimesForTopics() { producers.forEach((key, producerFuture) -> { if (producerFuture != null && producerFuture.isDone()) { Producer p = producerFuture.getNow(null); @@ -3220,65 +3218,17 @@ private void recordRateLimitMetrics(ConcurrentLongHashMap { - if (cnx.autoReadDisabledPublishBufferLimiting) { - cnx.autoReadDisabledPublishBufferLimiting = false; - cnx.enableCnxAutoRead(); - resumedConnections.increment(); - } - }); - - getBrokerService().resumedConnections(resumedConnections.intValue()); - } + PendingBytesPerThreadTracker.getInstance().decrementPublishBytes(msgSize, resumeThresholdPendingBytesPerThread); if (--pendingSendRequest == resumeReadsThreshold) { - enableCnxAutoRead(); + throttleTracker.setPendingSendRequestsExceeded(false); } + if (isNonPersistentTopic) { nonPersistentPendingMessages--; } } - @Override - public void enableCnxAutoRead() { - // we can add check (&& pendingSendRequest < MaxPendingSendRequests) here but then it requires - // pendingSendRequest to be volatile and it can be expensive while writing. also this will be called on if - // throttling is enable on the topic. so, avoid pendingSendRequest check will be fine. - if (ctx != null && !ctx.channel().config().isAutoRead() - && !autoReadDisabledRateLimiting && !autoReadDisabledPublishBufferLimiting) { - // Resume reading from socket if pending-request is not reached to threshold - ctx.channel().config().setAutoRead(true); - throttledConnections.dec(); - } - } - - @Override - public void disableCnxAutoRead() { - if (ctx != null && ctx.channel().config().isAutoRead()) { - ctx.channel().config().setAutoRead(false); - recordRateLimitMetrics(producers); - } - } - - @Override - public void cancelPublishRateLimiting() { - if (autoReadDisabledRateLimiting) { - autoReadDisabledRateLimiting = false; - } - } - - @Override - public void cancelPublishBufferLimiting() { - if (autoReadDisabledPublishBufferLimiting) { - autoReadDisabledPublishBufferLimiting = false; - throttledConnectionsGlobal.dec(); - } - } - private ServerError getErrorCode(CompletableFuture future) { return getErrorCodeWithErrorLog(future, false, null); } @@ -3429,11 +3379,6 @@ public String getProxyVersion() { return proxyVersion; } - @VisibleForTesting - void setAutoReadDisabledRateLimiting(boolean isLimiting) { - this.autoReadDisabledRateLimiting = isLimiting; - } - @Override public boolean isPreciseDispatcherFlowControl() { return preciseDispatcherFlowControl; @@ -3598,4 +3543,20 @@ protected AuthenticationState getOriginalAuthState() { protected void setAuthRole(String authRole) { this.authRole = authRole; } + + /** + * {@inheritDoc} + */ + @Override + public void incrementThrottleCount() { + throttleTracker.incrementThrottleCount(); + } + + /** + * {@inheritDoc} + */ + @Override + public void decrementThrottleCount() { + throttleTracker.decrementThrottleCount(); + } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java new file mode 100644 index 0000000000000..f223d6eee3795 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import io.prometheus.client.Gauge; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import lombok.extern.slf4j.Slf4j; + +/** + * Tracks the state of throttling for a connection. The throttling happens by pausing reads by setting + * Netty {@link io.netty.channel.ChannelConfig#setAutoRead(boolean)} to false for the channel (connection). + *

+ * There can be multiple rate limiters that can throttle a connection. Each rate limiter will independently + * call the {@link #incrementThrottleCount()} and {@link #decrementThrottleCount()} methods to signal that the + * connection should be throttled or not. The connection will be throttled if the counter is greater than 0. + *

+ * Besides the rate limiters, the connection can also be throttled if the number of pending publish requests exceeds + * a configured threshold. This throttling is toggled with the {@link #setPendingSendRequestsExceeded} method. + * There's also per-thread memory limits which could throttle the connection. This throttling is toggled with the + * {@link #setPublishBufferLimiting} method. Internally, these two methods will call the + * {@link #incrementThrottleCount()} and {@link #decrementThrottleCount()} methods when the state changes. + */ +@Slf4j +final class ServerCnxThrottleTracker { + private static final Gauge throttledConnections = Gauge.build() + .name("pulsar_broker_throttled_connections") + .help("Counter of connections throttled because of per-connection limit") + .register(); + + private static final AtomicIntegerFieldUpdater THROTTLE_COUNT_UPDATER = + AtomicIntegerFieldUpdater.newUpdater( + ServerCnxThrottleTracker.class, "throttleCount"); + + private static final AtomicIntegerFieldUpdater + PENDING_SEND_REQUESTS_EXCEEDED_UPDATER = + AtomicIntegerFieldUpdater.newUpdater( + ServerCnxThrottleTracker.class, "pendingSendRequestsExceeded"); + private static final AtomicIntegerFieldUpdater PUBLISH_BUFFER_LIMITING_UPDATER = + AtomicIntegerFieldUpdater.newUpdater( + ServerCnxThrottleTracker.class, "publishBufferLimiting"); + private final ServerCnx serverCnx; + private volatile int throttleCount; + private volatile int pendingSendRequestsExceeded; + private volatile int publishBufferLimiting; + + public ServerCnxThrottleTracker(ServerCnx serverCnx) { + this.serverCnx = serverCnx; + + } + + /** + * See {@link Producer#incrementThrottleCount()} for documentation. + */ + public void incrementThrottleCount() { + int currentThrottleCount = THROTTLE_COUNT_UPDATER.incrementAndGet(this); + if (currentThrottleCount == 1) { + changeAutoRead(false); + } + } + + /** + * See {@link Producer#decrementThrottleCount()} for documentation. + */ + public void decrementThrottleCount() { + int currentThrottleCount = THROTTLE_COUNT_UPDATER.decrementAndGet(this); + if (currentThrottleCount == 0) { + changeAutoRead(true); + } + } + + private void changeAutoRead(boolean autoRead) { + if (isChannelActive()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Setting auto read to {}", serverCnx.ctx().channel(), autoRead); + } + // change the auto read flag on the channel + serverCnx.ctx().channel().config().setAutoRead(autoRead); + } + // update the metrics that track throttling + if (autoRead) { + serverCnx.getBrokerService().resumedConnections(1); + } else if (isChannelActive()) { + serverCnx.increasePublishLimitedTimesForTopics(); + serverCnx.getBrokerService().pausedConnections(1); + } + } + + private boolean isChannelActive() { + return serverCnx.isActive() && serverCnx.ctx() != null && serverCnx.ctx().channel().isActive(); + } + + public void setPublishBufferLimiting(boolean throttlingEnabled) { + changeThrottlingFlag(PUBLISH_BUFFER_LIMITING_UPDATER, throttlingEnabled); + } + + public void setPendingSendRequestsExceeded(boolean throttlingEnabled) { + boolean changed = changeThrottlingFlag(PENDING_SEND_REQUESTS_EXCEEDED_UPDATER, throttlingEnabled); + if (changed) { + // update the metrics that track throttling due to pending send requests + if (throttlingEnabled) { + throttledConnections.inc(); + } else { + throttledConnections.dec(); + } + } + } + + private boolean changeThrottlingFlag(AtomicIntegerFieldUpdater throttlingFlagFieldUpdater, + boolean throttlingEnabled) { + // don't change a throttling flag if the channel is not active + if (!isChannelActive()) { + return false; + } + if (throttlingFlagFieldUpdater.compareAndSet(this, booleanToInt(!throttlingEnabled), + booleanToInt(throttlingEnabled))) { + if (throttlingEnabled) { + incrementThrottleCount(); + } else { + decrementThrottleCount(); + } + return true; + } else { + return false; + } + } + + private static int booleanToInt(boolean value) { + return value ? 1 : 0; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 9deeafdb272f5..61107b7b0dbb3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -26,6 +26,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; @@ -64,16 +65,18 @@ default long getNumberOfEntriesDelayed() { List getConsumers(); - CompletableFuture close(); - CompletableFuture delete(); CompletableFuture deleteForcefully(); - CompletableFuture disconnect(); + CompletableFuture disconnect(Optional assignedBrokerLookupData); + + CompletableFuture close(boolean disconnectConsumers, Optional assignedBrokerLookupData); CompletableFuture doUnsubscribe(Consumer consumer); + CompletableFuture doUnsubscribe(Consumer consumer, boolean forcefully); + CompletableFuture clearBacklog(); CompletableFuture skipMessages(int numMessagesToSkip); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index 244f982e59b58..343aef09c1c55 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -219,32 +219,12 @@ CompletableFuture close( void checkMessageDeduplicationInfo(); - void checkTopicPublishThrottlingRate(); - - void incrementPublishCount(int numOfMessages, long msgSizeInBytes); - - void resetTopicPublishCountAndEnableReadIfRequired(); - - void resetBrokerPublishCountAndEnableReadIfRequired(boolean doneReset); - - boolean isPublishRateExceeded(); - - boolean isTopicPublishRateExceeded(int msgSize, int numMessages); - - boolean isResourceGroupRateLimitingEnabled(); - - boolean isResourceGroupPublishRateExceeded(int msgSize, int numMessages); - - boolean isBrokerPublishRateExceeded(); + void incrementPublishCount(Producer producer, int numOfMessages, long msgSizeInBytes); boolean shouldProducerMigrate(); boolean isReplicationBacklogExist(); - void disableCnxAutoRead(); - - void enableCnxAutoRead(); - CompletableFuture onPoliciesUpdate(Policies data); CompletableFuture checkBacklogQuotaExceeded(String producerName, BacklogQuotaType backlogQuotaType); @@ -338,7 +318,7 @@ default boolean isSystemTopic() { boolean isPersistent(); - boolean isFenced(); + boolean isTransferring(); /* ------ Transaction related ------ */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransportCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransportCnx.java index c09d63a9232eb..6db085bd90a9b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransportCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransportCnx.java @@ -61,19 +61,11 @@ public interface TransportCnx { void closeProducer(Producer producer); void closeProducer(Producer producer, Optional assignedBrokerLookupData); - void cancelPublishRateLimiting(); - - void cancelPublishBufferLimiting(); - - void disableCnxAutoRead(); - - void enableCnxAutoRead(); - void execute(Runnable runnable); void removedConsumer(Consumer consumer); - void closeConsumer(Consumer consumer); + void closeConsumer(Consumer consumer, Optional assignedBrokerLookupData); boolean isPreciseDispatcherFlowControl(); @@ -93,4 +85,22 @@ public interface TransportCnx { * is null if the connection liveness check is disabled. */ CompletableFuture checkConnectionLiveness(); + + /** + * Increments the counter that controls the throttling of the connection by pausing reads. + * The connection will be throttled while the counter is greater than 0. + *

+ * The caller is responsible for decrementing the counter by calling {@link #decrementThrottleCount()} when the + * connection should no longer be throttled. + */ + void incrementThrottleCount(); + + /** + * Decrements the counter that controls the throttling of the connection by pausing reads. + * The connection will be throttled while the counter is greater than 0. + *

+ * This method should be called when the connection should no longer be throttled. However, the caller should have + * previously called {@link #incrementThrottleCount()}. + */ + void decrementThrottleCount(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java index c106b1603f6bd..29bca715741ad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java @@ -19,10 +19,12 @@ package org.apache.pulsar.broker.service.nonpersistent; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.bookkeeper.mledger.Entry; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; @@ -32,6 +34,7 @@ import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.stats.Rate; @@ -126,9 +129,12 @@ public synchronized boolean canUnsubscribe(Consumer consumer) { } @Override - public CompletableFuture close() { + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { IS_CLOSED_UPDATER.set(this, TRUE); - return disconnectAllConsumers(); + getRateLimiter().ifPresent(DispatchRateLimiter::close); + return disconnectConsumers + ? disconnectAllConsumers(false, assignedBrokerLookupData) : CompletableFuture.completedFuture(null); } @Override @@ -147,12 +153,13 @@ public synchronized void consumerFlow(Consumer consumer, int additionalNumberOfM } @Override - public synchronized CompletableFuture disconnectAllConsumers(boolean isResetCursor) { + public synchronized CompletableFuture disconnectAllConsumers( + boolean isResetCursor, Optional assignedBrokerLookupData) { closeFuture = new CompletableFuture<>(); if (consumerList.isEmpty()) { closeFuture.complete(null); } else { - consumerList.forEach(Consumer::disconnect); + consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); } return closeFuture; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java index 25e3e2894daa1..5e8eda2ab70e6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java @@ -101,11 +101,6 @@ protected void scheduleReadOnActiveConsumer() { // No-op } - @Override - protected void readMoreEntries(Consumer consumer) { - // No-op - } - @Override protected void cancelPendingRead() { // No-op diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index 6ec969c927a8c..92aba6221da73 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.AbstractSubscription; import org.apache.pulsar.broker.service.AnalyzeBacklogResult; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -277,40 +278,67 @@ public boolean isSubscriptionMigrated() { return topic.isMigrated(); } + /** + * Disconnect all consumers from this subscription. + * + * @return CompletableFuture indicating the completion of the operation. + */ @Override - public CompletableFuture close() { + public synchronized CompletableFuture disconnect(Optional assignedBrokerLookupData) { + CompletableFuture closeFuture = new CompletableFuture<>(); + + (dispatcher != null + ? dispatcher.disconnectAllConsumers(false, assignedBrokerLookupData) + : CompletableFuture.completedFuture(null)) + .thenRun(() -> { + log.info("[{}][{}] Successfully disconnected subscription consumers", topicName, subName); + closeFuture.complete(null); + }).exceptionally(exception -> { + log.error("[{}][{}] Error disconnecting subscription consumers", topicName, subName, exception); + closeFuture.completeExceptionally(exception); + return null; + }); + + return closeFuture; + + } + + private CompletableFuture fence() { IS_FENCED_UPDATER.set(this, TRUE); return CompletableFuture.completedFuture(null); } + /** - * Disconnect all consumers attached to the dispatcher and close this subscription. + * Fence this subscription and optionally disconnect all consumers. * - * @return CompletableFuture indicating the completion of disconnect operation + * @return CompletableFuture indicating the completion of the operation. */ @Override - public synchronized CompletableFuture disconnect() { - CompletableFuture disconnectFuture = new CompletableFuture<>(); + public synchronized CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { + CompletableFuture closeFuture = new CompletableFuture<>(); // block any further consumers on this subscription IS_FENCED_UPDATER.set(this, TRUE); - (dispatcher != null ? dispatcher.close() : CompletableFuture.completedFuture(null)).thenCompose(v -> close()) + (dispatcher != null + ? dispatcher.close(disconnectConsumers, assignedBrokerLookupData) + : CompletableFuture.completedFuture(null)) .thenRun(() -> { - log.info("[{}][{}] Successfully disconnected and closed subscription", topicName, subName); - disconnectFuture.complete(null); + log.info("[{}][{}] Successfully closed subscription", topicName, subName); + closeFuture.complete(null); }).exceptionally(exception -> { IS_FENCED_UPDATER.set(this, FALSE); if (dispatcher != null) { dispatcher.reset(); } - log.error("[{}][{}] Error disconnecting consumers from subscription", topicName, subName, - exception); - disconnectFuture.completeExceptionally(exception); + log.error("[{}][{}] Error closing subscription", topicName, subName, exception); + closeFuture.completeExceptionally(exception); return null; }); - return disconnectFuture; + return closeFuture; } /** @@ -349,7 +377,7 @@ private CompletableFuture delete(boolean closeIfConsumersConnected) { CompletableFuture closeSubscriptionFuture = new CompletableFuture<>(); if (closeIfConsumersConnected) { - this.disconnect().thenRun(() -> { + this.close(true, Optional.empty()).thenRun(() -> { closeSubscriptionFuture.complete(null); }).exceptionally(ex -> { log.error("[{}][{}] Error disconnecting and closing subscription", topicName, subName, ex); @@ -357,7 +385,7 @@ private CompletableFuture delete(boolean closeIfConsumersConnected) { return null; }); } else { - this.close().thenRun(() -> { + this.fence().thenRun(() -> { closeSubscriptionFuture.complete(null); }).exceptionally(exception -> { log.error("[{}][{}] Error closing subscription", topicName, subName, exception); @@ -401,11 +429,24 @@ private CompletableFuture delete(boolean closeIfConsumersConnected) { */ @Override public CompletableFuture doUnsubscribe(Consumer consumer) { + return doUnsubscribe(consumer, false); + } + + /** + * Handle unsubscribe command from the client API Check with the dispatcher is this consumer can proceed with + * unsubscribe. + * + * @param consumer consumer object that is initiating the unsubscribe operation + * @param force unsubscribe forcefully by disconnecting consumers and closing subscription + * @return CompletableFuture indicating the completion of ubsubscribe operation + */ + @Override + public CompletableFuture doUnsubscribe(Consumer consumer, boolean force) { CompletableFuture future = new CompletableFuture<>(); try { - if (dispatcher.canUnsubscribe(consumer)) { + if (force || dispatcher.canUnsubscribe(consumer)) { consumer.close(); - return delete(); + return delete(force); } future.completeExceptionally( new ServerMetadataException("Unconnected or shared consumer attempting to unsubscribe")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 6589e7e1ec79c..00cf3a6583b9a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -422,7 +422,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c List> futures = new ArrayList<>(); replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); producers.values().forEach(producer -> futures.add(producer.disconnect())); - subscriptions.forEach((s, sub) -> futures.add(sub.disconnect())); + subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); FutureUtil.waitForAll(futures).thenRun(() -> { closeClientFuture.complete(null); }).exceptionally(ex -> { @@ -507,6 +507,9 @@ public CompletableFuture close( lock.writeLock().lock(); try { + if (!disconnectClients) { + transferring = true; + } if (!isFenced || closeWithoutWaitingClientDisconnect) { isFenced = true; } else { @@ -523,16 +526,21 @@ public CompletableFuture close( replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( - brokerService.getPulsar(), topic).thenAccept(lookupData -> - producers.values().forEach(producer -> futures.add(producer.disconnect(lookupData))) + brokerService.getPulsar(), topic).thenAccept(lookupData -> { + producers.values().forEach(producer -> futures.add(producer.disconnect(lookupData))); + // Topics unloaded due to the ExtensibleLoadManager undergo closing twice: first with + // disconnectClients = false, second with disconnectClients = true. The check below identifies the + // cases when Topic.close is called outside the scope of the ExtensibleLoadManager. In these + // situations, we must pursue the regular Subscription.close, as Topic.close is invoked just once. + if (isTransferring()) { + subscriptions.forEach((s, sub) -> futures.add(sub.disconnect(lookupData))); + } else { + subscriptions.forEach((s, sub) -> futures.add(sub.close(true, lookupData))); + } + } )); - } - if (topicPublishRateLimiter != null) { - topicPublishRateLimiter.close(); - } - subscriptions.forEach((s, sub) -> futures.add(sub.disconnect())); - if (this.resourceGroupPublishLimiter != null) { - this.resourceGroupPublishLimiter.unregisterRateLimitFunction(this.getName()); + } else { + subscriptions.forEach((s, sub) -> futures.add(sub.close(false, Optional.empty()))); } if (entryFilters != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index 233972452e97f..b29cbcd660db1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -21,19 +21,17 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.util.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DispatchRateLimiter { - public enum Type { TOPIC, SUBSCRIPTION, @@ -47,8 +45,8 @@ public enum Type { private final Type type; private final BrokerService brokerService; - private RateLimiter dispatchRateLimiterOnMessage; - private RateLimiter dispatchRateLimiterOnByte; + private volatile AsyncTokenBucket dispatchRateLimiterOnMessage; + private volatile AsyncTokenBucket dispatchRateLimiterOnByte; public DispatchRateLimiter(PersistentTopic topic, Type type) { this(topic, null, type); @@ -78,7 +76,7 @@ public DispatchRateLimiter(BrokerService brokerService) { * @return */ public long getAvailableDispatchRateLimitOnMsg() { - return dispatchRateLimiterOnMessage == null ? -1 : dispatchRateLimiterOnMessage.getAvailablePermits(); + return dispatchRateLimiterOnMessage == null ? -1 : Math.max(dispatchRateLimiterOnMessage.getTokens(), 0); } /** @@ -87,32 +85,22 @@ public long getAvailableDispatchRateLimitOnMsg() { * @return */ public long getAvailableDispatchRateLimitOnByte() { - return dispatchRateLimiterOnByte == null ? -1 : dispatchRateLimiterOnByte.getAvailablePermits(); + return dispatchRateLimiterOnByte == null ? -1 : Math.max(dispatchRateLimiterOnByte.getTokens(), 0); } /** * It acquires msg and bytes permits from rate-limiter and returns if acquired permits succeed. * - * @param msgPermits - * @param bytePermits - * @return - */ - public boolean tryDispatchPermit(long msgPermits, long bytePermits) { - boolean acquiredMsgPermit = msgPermits <= 0 || dispatchRateLimiterOnMessage == null - || dispatchRateLimiterOnMessage.tryAcquire(msgPermits); - boolean acquiredBytePermit = bytePermits <= 0 || dispatchRateLimiterOnByte == null - || dispatchRateLimiterOnByte.tryAcquire(bytePermits); - return acquiredMsgPermit && acquiredBytePermit; - } - - /** - * checks if dispatch-rate limit is configured and if it's configured then check if permits are available or not. - * - * @return + * @param numberOfMessages + * @param byteSize */ - public boolean hasMessageDispatchPermit() { - return (dispatchRateLimiterOnMessage == null || dispatchRateLimiterOnMessage.getAvailablePermits() > 0) - && (dispatchRateLimiterOnByte == null || dispatchRateLimiterOnByte.getAvailablePermits() > 0); + public void consumeDispatchQuota(long numberOfMessages, long byteSize) { + if (numberOfMessages > 0 && dispatchRateLimiterOnMessage != null) { + dispatchRateLimiterOnMessage.consumeTokens(numberOfMessages); + } + if (byteSize > 0 && dispatchRateLimiterOnByte != null) { + dispatchRateLimiterOnByte.consumeTokens(byteSize); + } } /** @@ -227,60 +215,40 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { long msgRate = dispatchRate.getDispatchThrottlingRateInMsg(); long byteRate = dispatchRate.getDispatchThrottlingRateInByte(); - long ratePeriod = dispatchRate.getRatePeriodInSecond(); + long ratePeriodNanos = TimeUnit.SECONDS.toNanos(Math.max(dispatchRate.getRatePeriodInSecond(), 1)); - Supplier permitUpdaterMsg = dispatchRate.isRelativeToPublishRate() - ? () -> getRelativeDispatchRateInMsg(dispatchRate) - : null; // update msg-rateLimiter if (msgRate > 0) { - if (this.dispatchRateLimiterOnMessage == null) { + if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnMessage = - RateLimiter.builder() - .scheduledExecutorService(brokerService.pulsar().getExecutor()) - .permits(msgRate) - .rateTime(ratePeriod) - .timeUnit(TimeUnit.SECONDS) - .permitUpdater(permitUpdaterMsg) - .isDispatchOrPrecisePublishRateLimiter(true) + AsyncTokenBucket.builderForDynamicRate() + .rateFunction(() -> getRelativeDispatchRateInMsg(dispatchRate)) + .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { - this.dispatchRateLimiterOnMessage.setRate(msgRate, dispatchRate.getRatePeriodInSecond(), - TimeUnit.SECONDS, permitUpdaterMsg); + this.dispatchRateLimiterOnMessage = + AsyncTokenBucket.builder().rate(msgRate).ratePeriodNanos(ratePeriodNanos) + .build(); } } else { - // message-rate should be disable and close - if (this.dispatchRateLimiterOnMessage != null) { - this.dispatchRateLimiterOnMessage.close(); - this.dispatchRateLimiterOnMessage = null; - } + this.dispatchRateLimiterOnMessage = null; } - Supplier permitUpdaterByte = dispatchRate.isRelativeToPublishRate() - ? () -> getRelativeDispatchRateInByte(dispatchRate) - : null; // update byte-rateLimiter if (byteRate > 0) { - if (this.dispatchRateLimiterOnByte == null) { + if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnByte = - RateLimiter.builder() - .scheduledExecutorService(brokerService.pulsar().getExecutor()) - .permits(byteRate) - .rateTime(ratePeriod) - .timeUnit(TimeUnit.SECONDS) - .permitUpdater(permitUpdaterByte) - .isDispatchOrPrecisePublishRateLimiter(true) + AsyncTokenBucket.builderForDynamicRate() + .rateFunction(() -> getRelativeDispatchRateInByte(dispatchRate)) + .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { - this.dispatchRateLimiterOnByte.setRate(byteRate, dispatchRate.getRatePeriodInSecond(), - TimeUnit.SECONDS, permitUpdaterByte); + this.dispatchRateLimiterOnByte = + AsyncTokenBucket.builder().rate(byteRate).ratePeriodNanos(ratePeriodNanos) + .build(); } } else { - // message-rate should be disable and close - if (this.dispatchRateLimiterOnByte != null) { - this.dispatchRateLimiterOnByte.close(); - this.dispatchRateLimiterOnByte = null; - } + this.dispatchRateLimiterOnByte = null; } } @@ -323,11 +291,9 @@ public static boolean isDispatchRateEnabled(DispatchRate dispatchRate) { public void close() { // close rate-limiter if (dispatchRateLimiterOnMessage != null) { - dispatchRateLimiterOnMessage.close(); dispatchRateLimiterOnMessage = null; } if (dispatchRateLimiterOnByte != null) { - dispatchRateLimiterOnByte.close(); dispatchRateLimiterOnByte = null; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 08882982297ab..191b19e317163 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -148,10 +148,7 @@ protected boolean replicateEntries(List entries) { continue; } - dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.tryDispatchPermit(1, entry.getLength())); - - msgOut.recordEvent(headersAndPayload.readableBytes()); - + dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.consumeDispatchQuota(1, entry.getLength())); msg.setReplicatedFrom(localCluster); headersAndPayload.retain(); @@ -181,6 +178,7 @@ protected boolean replicateEntries(List entries) { msg.setSchemaInfoForReplicator(schemaFuture.get()); msg.getMessageBuilder().clearTxnidMostBits(); msg.getMessageBuilder().clearTxnidLeastBits(); + msgOut.recordEvent(headersAndPayload.readableBytes()); // Increment pending messages for messages produced locally PENDING_MESSAGES_UPDATER.incrementAndGet(this); producer.sendAsync(msg, ProducerSendCallback.create(this, entry, msg)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index 238dc740509b1..802dd91796127 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -474,6 +474,10 @@ private boolean isDeduplicationEnabled() { * Topic will call this method whenever a producer connects. */ public void producerAdded(String producerName) { + if (!isEnabled()) { + return; + } + // Producer is no-longer inactive inactiveProducers.remove(producerName); } @@ -482,6 +486,10 @@ public void producerAdded(String producerName) { * Topic will call this method whenever a producer disconnects. */ public void producerRemoved(String producerName) { + if (!isEnabled()) { + return; + } + // Producer is no-longer active inactiveProducers.put(producerName, System.currentTimeMillis()); } @@ -493,6 +501,14 @@ public synchronized void purgeInactiveProducers() { long minimumActiveTimestamp = System.currentTimeMillis() - TimeUnit.MINUTES .toMillis(pulsar.getConfiguration().getBrokerDeduplicationProducerInactivityTimeoutMinutes()); + // if not enabled just clear all inactive producer record. + if (!isEnabled()) { + if (!inactiveProducers.isEmpty()) { + inactiveProducers.clear(); + } + return; + } + Iterator> mapIterator = inactiveProducers.entrySet().iterator(); boolean hasInactive = false; while (mapIterator.hasNext()) { @@ -545,5 +561,10 @@ ManagedCursor getManagedCursor() { return managedCursor; } + @VisibleForTesting + Map getInactiveProducers() { + return inactiveProducers; + } + private static final Logger log = LoggerFactory.getLogger(MessageDeduplication.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index b3d48252efe58..cafc398a3c843 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -51,6 +51,7 @@ import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; @@ -275,6 +276,10 @@ public synchronized void readMoreEntries() { if (shouldPauseDeliveryForDelayTracker()) { return; } + if (topic.isTransferring()) { + // Do not deliver messages for topics that are undergoing transfer, as the acknowledgments would be ignored. + return; + } // totalAvailablePermits may be updated by other threads int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); @@ -397,52 +402,52 @@ protected Pair calculateToRead(int currentTotalAvailablePermits) if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { if (topic.getBrokerDispatchRateLimiter().isPresent()) { DispatchRateLimiter brokerRateLimiter = topic.getBrokerDispatchRateLimiter().get(); - if (reachDispatchRateLimit(brokerRateLimiter)) { + Pair calculateToRead = + updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded broker message-rate {}/{}, schedule after a {}", name, brokerRateLimiter.getDispatchRateOnMsg(), brokerRateLimiter.getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } + reScheduleRead(); return Pair.of(-1, -1L); - } else { - Pair calculateToRead = - updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); - messagesToRead = calculateToRead.getLeft(); - bytesToRead = calculateToRead.getRight(); } } if (topic.getDispatchRateLimiter().isPresent()) { DispatchRateLimiter topicRateLimiter = topic.getDispatchRateLimiter().get(); - if (reachDispatchRateLimit(topicRateLimiter)) { + Pair calculateToRead = + updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded topic message-rate {}/{}, schedule after a {}", name, topicRateLimiter.getDispatchRateOnMsg(), topicRateLimiter.getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } + reScheduleRead(); return Pair.of(-1, -1L); - } else { - Pair calculateToRead = - updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); - messagesToRead = calculateToRead.getLeft(); - bytesToRead = calculateToRead.getRight(); } } if (dispatchRateLimiter.isPresent()) { - if (reachDispatchRateLimit(dispatchRateLimiter.get())) { + Pair calculateToRead = + updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded subscription message-rate {}/{}, schedule after a {}", name, dispatchRateLimiter.get().getDispatchRateOnMsg(), dispatchRateLimiter.get().getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } + reScheduleRead(); return Pair.of(-1, -1L); - } else { - Pair calculateToRead = - updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); - messagesToRead = calculateToRead.getLeft(); - bytesToRead = calculateToRead.getRight(); } } } @@ -484,7 +489,8 @@ public synchronized boolean canUnsubscribe(Consumer consumer) { } @Override - public CompletableFuture close() { + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { IS_CLOSED_UPDATER.set(this, TRUE); Optional delayedDeliveryTracker; @@ -494,19 +500,20 @@ public CompletableFuture close() { } delayedDeliveryTracker.ifPresent(DelayedDeliveryTracker::close); - dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); - return disconnectAllConsumers(); + return disconnectConsumers + ? disconnectAllConsumers(false, assignedBrokerLookupData) : CompletableFuture.completedFuture(null); } @Override - public synchronized CompletableFuture disconnectAllConsumers(boolean isResetCursor) { + public synchronized CompletableFuture disconnectAllConsumers( + boolean isResetCursor, Optional assignedBrokerLookupData) { closeFuture = new CompletableFuture<>(); if (consumerList.isEmpty()) { closeFuture.complete(null); } else { - consumerList.forEach(consumer -> consumer.disconnect(isResetCursor)); + consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); cancelPendingRead(); } return closeFuture; @@ -665,15 +672,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis long totalEntries = 0; int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; - int firstAvailableConsumerPermits, currentTotalAvailablePermits; - boolean dispatchMessage; - while (entriesToDispatch > 0) { - firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); - currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); - dispatchMessage = currentTotalAvailablePermits > 0 && firstAvailableConsumerPermits > 0; - if (!dispatchMessage) { - break; - } + // If the dispatcher is closed, firstAvailableConsumerPermits will be 0, which skips dispatching the + // messages. + while (entriesToDispatch > 0 && isAtleastOneConsumerAvailable()) { Consumer c = getNextConsumer(); if (c == null) { // Do nothing, cursor will be rewind at reconnection @@ -1025,13 +1026,6 @@ public Optional getRateLimiter() { return dispatchRateLimiter; } - @Override - public void updateRateLimiter() { - if (!initializeDispatchRateLimiterIfNeeded()) { - this.dispatchRateLimiter.ifPresent(DispatchRateLimiter::updateDispatchRate); - } - } - @Override public boolean initializeDispatchRateLimiterIfNeeded() { if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter.isDispatchRateEnabled( diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index 5e9183df0b1df..806773af45189 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Objects; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -154,7 +153,7 @@ public void readEntriesComplete(final List entries, Object obj) { executor.execute(() -> internalReadEntriesComplete(entries, obj)); } - public synchronized void internalReadEntriesComplete(final List entries, Object obj) { + private synchronized void internalReadEntriesComplete(final List entries, Object obj) { ReadEntriesCtx readEntriesCtx = (ReadEntriesCtx) obj; Consumer readConsumer = readEntriesCtx.getConsumer(); long epoch = readEntriesCtx.getEpoch(); @@ -194,11 +193,17 @@ public synchronized void internalReadEntriesComplete(final List entries, } } - if (currentConsumer == null || readConsumer != currentConsumer) { - // Active consumer has changed since the read request has been issued. We need to rewind the cursor and - // re-issue the read request for the new consumer + if (currentConsumer == null || readConsumer != currentConsumer || topic.isTransferring()) { + // Active consumer has changed since the read request has been issued, or the topic is being transferred to + // another broker. We need to rewind the cursor and re-issue the read request for the new consumer. if (log.isDebugEnabled()) { - log.debug("[{}] rewind because no available consumer found", name); + if (currentConsumer == null) { + log.debug("[{}] rewind because no available consumer found", name); + } else if (readConsumer != currentConsumer) { + log.debug("[{}] rewind because active consumer changed", name); + } else { + log.debug("[{}] rewind because topic is transferring", name); + } } entries.forEach(Entry::release); cursor.rewind(); @@ -309,8 +314,7 @@ public void redeliverUnacknowledgedMessages(Consumer consumer, List 0) { synchronized (this) { @@ -406,52 +416,52 @@ protected Pair calculateToRead(Consumer consumer) { if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { if (topic.getBrokerDispatchRateLimiter().isPresent()) { DispatchRateLimiter brokerRateLimiter = topic.getBrokerDispatchRateLimiter().get(); - if (reachDispatchRateLimit(brokerRateLimiter)) { + Pair calculateToRead = + updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded broker message-rate {}/{}, schedule after a {}", name, brokerRateLimiter.getDispatchRateOnMsg(), brokerRateLimiter.getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } + reScheduleRead(); return Pair.of(-1, -1L); - } else { - Pair calculateToRead = - updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); - messagesToRead = calculateToRead.getLeft(); - bytesToRead = calculateToRead.getRight(); } } if (topic.getDispatchRateLimiter().isPresent()) { DispatchRateLimiter topicRateLimiter = topic.getDispatchRateLimiter().get(); - if (reachDispatchRateLimit(topicRateLimiter)) { + Pair calculateToRead = + updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded topic message-rate {}/{}, schedule after a {}", name, topicRateLimiter.getDispatchRateOnMsg(), topicRateLimiter.getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } + reScheduleRead(); return Pair.of(-1, -1L); - } else { - Pair calculateToRead = - updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); - messagesToRead = calculateToRead.getLeft(); - bytesToRead = calculateToRead.getRight(); } } if (dispatchRateLimiter.isPresent()) { - if (reachDispatchRateLimit(dispatchRateLimiter.get())) { + Pair calculateToRead = + updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded subscription message-rate {}/{}, schedule after a {}", name, dispatchRateLimiter.get().getDispatchRateOnMsg(), dispatchRateLimiter.get().getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } + reScheduleRead(); return Pair.of(-1, -1L); - } else { - Pair calculateToRead = - updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); - messagesToRead = calculateToRead.getLeft(); - bytesToRead = calculateToRead.getRight(); } } } @@ -549,13 +559,6 @@ public Optional getRateLimiter() { return dispatchRateLimiter; } - @Override - public void updateRateLimiter() { - if (!initializeDispatchRateLimiterIfNeeded()) { - this.dispatchRateLimiter.ifPresent(DispatchRateLimiter::updateDispatchRate); - } - } - @Override public boolean initializeDispatchRateLimiterIfNeeded() { if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter.isDispatchRateEnabled( @@ -567,13 +570,6 @@ public boolean initializeDispatchRateLimiterIfNeeded() { return false; } - @Override - public CompletableFuture close() { - IS_CLOSED_UPDATER.set(this, TRUE); - dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); - return disconnectAllConsumers(); - } - @Override public boolean checkAndUnblockIfStuck() { Consumer consumer = ACTIVE_CONSUMER_UPDATER.get(this); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index a96036b7cfe2a..a4ac52d5ded7d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -201,8 +201,11 @@ private int getAvailablePermits() { // handle rate limit if (dispatchRateLimiter.isPresent() && dispatchRateLimiter.get().isDispatchRateLimitingEnabled()) { DispatchRateLimiter rateLimiter = dispatchRateLimiter.get(); + // if dispatch-rate is in msg then read only msg according to available permit + long availablePermitsOnMsg = rateLimiter.getAvailableDispatchRateLimitOnMsg(); + long availablePermitsOnByte = rateLimiter.getAvailableDispatchRateLimitOnByte(); // no permits from rate limit - if (!rateLimiter.hasMessageDispatchPermit()) { + if (availablePermitsOnByte == 0 || availablePermitsOnMsg == 0) { if (log.isDebugEnabled()) { log.debug("[{}] message-read exceeded topic replicator message-rate {}/{}," + " schedule after a {}", @@ -213,9 +216,6 @@ private int getAvailablePermits() { } return -1; } - - // if dispatch-rate is in msg then read only msg according to available permit - long availablePermitsOnMsg = rateLimiter.getAvailableDispatchRateLimitOnMsg(); if (availablePermitsOnMsg > 0) { availablePermits = Math.min(availablePermits, (int) availablePermitsOnMsg); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 0397eef8aa86c..dc79146110f00 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -57,6 +57,7 @@ import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.AbstractSubscription; import org.apache.pulsar.broker.service.AnalyzeBacklogResult; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -307,8 +308,9 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor topic.getManagedLedger().removeWaitingCursor(cursor); if (!cursor.isDurable()) { - // If cursor is not durable, we need to clean up the subscription as well - this.close().thenRun(() -> { + // If cursor is not durable, we need to clean up the subscription as well. No need to check for active + // consumers since we already validated that there are no consumers on this dispatcher. + this.closeCursor(false).thenRun(() -> { synchronized (this) { if (dispatcher != null) { dispatcher.close().thenRun(() -> { @@ -885,49 +887,77 @@ public int getTotalNonContiguousDeletedMessagesRange() { } /** - * Close the cursor ledger for this subscription. Requires that there are no active consumers on the dispatcher + * Close the cursor ledger for this subscription. Optionally verifies that there are no active consumers on the + * dispatcher. * - * @return CompletableFuture indicating the completion of delete operation + * @return CompletableFuture indicating the completion of close operation */ - @Override - public CompletableFuture close() { - synchronized (this) { - if (dispatcher != null && dispatcher.isConsumerConnected()) { - return FutureUtil.failedFuture(new SubscriptionBusyException("Subscription has active consumers")); - } - return this.pendingAckHandle.closeAsync().thenAccept(v -> { - IS_FENCED_UPDATER.set(this, TRUE); - log.info("[{}][{}] Successfully closed subscription [{}]", topicName, subName, cursor); - }); + private synchronized CompletableFuture closeCursor(boolean checkActiveConsumers) { + if (checkActiveConsumers && dispatcher != null && dispatcher.isConsumerConnected()) { + return FutureUtil.failedFuture(new SubscriptionBusyException("Subscription has active consumers")); } + return this.pendingAckHandle.closeAsync().thenAccept(v -> { + IS_FENCED_UPDATER.set(this, TRUE); + log.info("[{}][{}] Successfully closed subscription [{}]", topicName, subName, cursor); + }); } + /** - * Disconnect all consumers attached to the dispatcher and close this subscription. + * Disconnect all consumers from this subscription. * - * @return CompletableFuture indicating the completion of disconnect operation + * @return CompletableFuture indicating the completion of the operation. */ @Override - public synchronized CompletableFuture disconnect() { - if (fenceFuture != null){ + public synchronized CompletableFuture disconnect(Optional assignedBrokerLookupData) { + CompletableFuture disconnectFuture = new CompletableFuture<>(); + + (dispatcher != null + ? dispatcher.disconnectAllConsumers(false, assignedBrokerLookupData) + : CompletableFuture.completedFuture(null)) + .thenRun(() -> { + log.info("[{}][{}] Successfully disconnected subscription consumers", topicName, subName); + disconnectFuture.complete(null); + }).exceptionally(exception -> { + log.error("[{}][{}] Error disconnecting subscription consumers", topicName, subName, exception); + disconnectFuture.completeExceptionally(exception); + return null; + }); + + return disconnectFuture; + } + + /** + * Fence this subscription and optionally disconnect all consumers. + * + * @return CompletableFuture indicating the completion of the operation. + */ + @Override + public synchronized CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { + if (fenceFuture != null) { return fenceFuture; } + fenceFuture = new CompletableFuture<>(); // block any further consumers on this subscription IS_FENCED_UPDATER.set(this, TRUE); - (dispatcher != null ? dispatcher.close() : CompletableFuture.completedFuture(null)) - .thenCompose(v -> close()).thenRun(() -> { - log.info("[{}][{}] Successfully disconnected and closed subscription", topicName, subName); + (dispatcher != null + ? dispatcher.close(disconnectConsumers, assignedBrokerLookupData) + : CompletableFuture.completedFuture(null)) + // checkActiveConsumers is false since we just closed all of them if we wanted. + .thenCompose(__ -> closeCursor(false)).thenRun(() -> { + log.info("[{}][{}] Successfully closed the subscription", topicName, subName); fenceFuture.complete(null); }).exceptionally(exception -> { - log.error("[{}][{}] Error disconnecting consumers from subscription", topicName, subName, - exception); + log.error("[{}][{}] Error closing the subscription", topicName, subName, exception); fenceFuture.completeExceptionally(exception); resumeAfterFence(); return null; }); + return fenceFuture; } @@ -935,7 +965,7 @@ public synchronized CompletableFuture disconnect() { * Resume subscription after topic deletion or close failure. */ public synchronized void resumeAfterFence() { - // If "fenceFuture" is null, it means that "disconnect" has never been called. + // If "fenceFuture" is null, it means that "close" has never been called. if (fenceFuture != null) { fenceFuture.whenComplete((ignore, ignoreEx) -> { synchronized (PersistentSubscription.this) { @@ -992,7 +1022,7 @@ private CompletableFuture delete(boolean closeIfConsumersConnected) { CompletableFuture closeSubscriptionFuture = new CompletableFuture<>(); if (closeIfConsumersConnected) { - this.disconnect().thenRun(() -> { + this.close(true, Optional.empty()).thenRun(() -> { closeSubscriptionFuture.complete(null); }).exceptionally(ex -> { log.error("[{}][{}] Error disconnecting and closing subscription", topicName, subName, ex); @@ -1000,7 +1030,7 @@ private CompletableFuture delete(boolean closeIfConsumersConnected) { return null; }); } else { - this.close().thenRun(() -> { + this.closeCursor(true).thenRun(() -> { closeSubscriptionFuture.complete(null); }).exceptionally(exception -> { log.error("[{}][{}] Error closing subscription", topicName, subName, exception); @@ -1044,11 +1074,27 @@ private CompletableFuture delete(boolean closeIfConsumersConnected) { */ @Override public CompletableFuture doUnsubscribe(Consumer consumer) { + return doUnsubscribe(consumer, false); + } + + /** + * Handle unsubscribe command from the client API Check with the dispatcher is this consumer can proceed with + * unsubscribe. + * + * @param consumer consumer object that is initiating the unsubscribe operation + * @param force unsubscribe forcefully by disconnecting consumers and closing subscription + * @return CompletableFuture indicating the completion of unsubscribe operation + */ + @Override + public CompletableFuture doUnsubscribe(Consumer consumer, boolean force) { CompletableFuture future = new CompletableFuture<>(); try { - if (dispatcher.canUnsubscribe(consumer)) { + if (force || dispatcher.canUnsubscribe(consumer)) { + if (log.isDebugEnabled()) { + log.debug("[{}] unsubscribing forcefully {}-{}", topicName, subName, consumer.consumerName()); + } consumer.close(); - return delete(); + return delete(force); } future.completeExceptionally( new ServerMetadataException("Unconnected or shared consumer attempting to unsubscribe")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 1e57debef0c33..48069cf555448 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -313,8 +313,6 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS .build(); this.backloggedCursorThresholdEntries = brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold(); - registerTopicPolicyListener(); - this.messageDeduplication = new MessageDeduplication(brokerService.pulsar(), this, ledger); if (ledger.getProperties().containsKey(TOPIC_EPOCH_PROPERTY_NAME)) { topicEpoch = Optional.of(Long.parseLong(ledger.getProperties().get(TOPIC_EPOCH_PROPERTY_NAME))); @@ -327,7 +325,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS this.transactionBuffer = brokerService.getPulsar() .getTransactionBufferProvider().newTransactionBuffer(this); } else { - this.transactionBuffer = new TransactionBufferDisable(); + this.transactionBuffer = new TransactionBufferDisable(this); } transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry()); if (ledger instanceof ShadowManagedLedgerImpl) { @@ -420,7 +418,7 @@ public CompletableFuture initialize() { this.transactionBuffer = brokerService.getPulsar() .getTransactionBufferProvider().newTransactionBuffer(this); } else { - this.transactionBuffer = new TransactionBufferDisable(); + this.transactionBuffer = new TransactionBufferDisable(this); } shadowSourceTopic = null; } @@ -477,7 +475,7 @@ public CompletableFuture unloadSubscription(@Nonnull String subName) { new UnsupportedSubscriptionException(String.format("Unsupported subscription: %s", subName))); } // Fence old subscription -> Rewind cursor -> Replace with a new subscription. - return sub.disconnect().thenCompose(ignore -> { + return sub.close(true, Optional.empty()).thenCompose(ignore -> { if (!lock.writeLock().tryLock()) { return CompletableFuture.failedFuture(new SubscriptionConflictUnloadException(String.format("Conflict" + " topic-close, topic-delete, another-subscribe-unload, cannot unload subscription %s now", @@ -529,8 +527,7 @@ public void publishMessage(ByteBuf headersAndPayload, PublishContext publishCont return; } if (isExceedMaximumMessageSize(headersAndPayload.readableBytes(), publishContext)) { - publishContext.completed(new NotAllowedException("Exceed maximum message size") - , -1, -1); + publishContext.completed(new NotAllowedException("Exceed maximum message size"), -1, -1); decrementPendingWriteOpsAndCheck(); return; } @@ -651,6 +648,19 @@ public void addComplete(Position pos, ByteBuf entryData, Object ctx) { @Override public synchronized void addFailed(ManagedLedgerException exception, Object ctx) { + /* If the topic is being transferred(in the Releasing bundle state), + we don't want to forcefully close topic here. + Instead, we will rely on the service unit state channel's bundle(topic) transfer protocol. + At the end of the transfer protocol, at Owned state, the source broker should close the topic properly. + */ + if (transferring) { + if (log.isDebugEnabled()) { + log.debug("[{}] Failed to persist msg in store: {} while transferring.", + topic, exception.getMessage(), exception); + } + return; + } + PublishContext callback = (PublishContext) ctx; if (exception instanceof ManagedLedgerFencedException) { // If the managed ledger has been fenced, we cannot continue using it. We need to close and reopen @@ -1348,7 +1358,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, CompletableFuture closeClientFuture = new CompletableFuture<>(); List> futures = new ArrayList<>(); - subscriptions.forEach((s, sub) -> futures.add(sub.disconnect())); + subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); if (closeIfClientsConnected) { replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); @@ -1473,6 +1483,9 @@ public CompletableFuture close( lock.writeLock().lock(); try { + if (!disconnectClients) { + transferring = true; + } // closing managed-ledger waits until all producers/consumers/replicators get closed. Sometimes, broker // forcefully wants to close managed-ledger without waiting all resources to be closed. if (!isClosingOrDeleting || closeWithoutWaitingClientDisconnect) { @@ -1493,16 +1506,21 @@ public CompletableFuture close( shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( - brokerService.getPulsar(), topic).thenAccept(lookupData -> - producers.values().forEach(producer -> futures.add(producer.disconnect(lookupData))) + brokerService.getPulsar(), topic).thenAccept(lookupData -> { + producers.values().forEach(producer -> futures.add(producer.disconnect(lookupData))); + // Topics unloaded due to the ExtensibleLoadManager undergo closing twice: first with + // disconnectClients = false, second with disconnectClients = true. The check below identifies the + // cases when Topic.close is called outside the scope of the ExtensibleLoadManager. In these + // situations, we must pursue the regular Subscription.close, as Topic.close is invoked just once. + if (isTransferring()) { + subscriptions.forEach((s, sub) -> futures.add(sub.disconnect(lookupData))); + } else { + subscriptions.forEach((s, sub) -> futures.add(sub.close(true, lookupData))); + } + } )); - } - if (topicPublishRateLimiter != null) { - topicPublishRateLimiter.close(); - } - subscriptions.forEach((s, sub) -> futures.add(sub.disconnect())); - if (this.resourceGroupPublishLimiter != null) { - this.resourceGroupPublishLimiter.unregisterRateLimitFunction(this.getName()); + } else { + subscriptions.forEach((s, sub) -> futures.add(sub.close(false, Optional.empty()))); } //close entry filters @@ -1632,6 +1650,11 @@ public CompletableFuture checkReplication() { } List configuredClusters = topicPolicies.getReplicationClusters().get(); + if (CollectionUtils.isEmpty(configuredClusters)) { + log.warn("[{}] No replication clusters configured", name); + return CompletableFuture.completedFuture(null); + } + int newMessageTTLInSeconds = topicPolicies.getMessageTTLInSeconds().get(); String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); @@ -3754,6 +3777,8 @@ private CompletableFuture updateSubscriptionsDispatcherRateLimiter() { protected CompletableFuture initTopicPolicy() { if (brokerService.pulsar().getConfig().isSystemTopicEnabled() && brokerService.pulsar().getConfig().isTopicLevelPoliciesEnabled()) { + brokerService.getPulsar().getTopicPoliciesService() + .registerListener(TopicName.getPartitionedTopicName(topic), this); return CompletableFuture.completedFuture(null).thenRunAsync(() -> onUpdate( brokerService.getPulsar().getTopicPoliciesService() .getTopicPoliciesIfExists(TopicName.getPartitionedTopicName(topic))), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java index fb306348bcdbb..493072eb0c837 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java @@ -101,7 +101,7 @@ protected boolean replicateEntries(List entries) { continue; } - dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.tryDispatchPermit(1, entry.getLength())); + dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.consumeDispatchQuota(1, entry.getLength())); msgOut.recordEvent(headersAndPayload.readableBytes()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index 58f099a9dab80..b1de10e73b76f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -19,40 +19,32 @@ package org.apache.pulsar.broker.service.persistent; -import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.base.MoreObjects; import java.util.Objects; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.SubscribeRate; -import org.apache.pulsar.common.util.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SubscribeRateLimiter { - private final String topicName; private final BrokerService brokerService; - private ConcurrentHashMap subscribeRateLimiter; - private final ScheduledExecutorService executorService; - private ScheduledFuture resetTask; + private ConcurrentHashMap subscribeRateLimiter; private SubscribeRate subscribeRate; public SubscribeRateLimiter(PersistentTopic topic) { this.topicName = topic.getName(); this.brokerService = topic.getBrokerService(); subscribeRateLimiter = new ConcurrentHashMap<>(); - this.executorService = brokerService.pulsar().getExecutor(); // get subscribeRate from topic level policies this.subscribeRate = topic.getSubscribeRate(); if (isSubscribeRateEnabled(this.subscribeRate)) { - resetTask = createTask(); log.info("[{}] configured subscribe-dispatch rate at broker {}", this.topicName, subscribeRate); } } @@ -64,7 +56,7 @@ public SubscribeRateLimiter(PersistentTopic topic) { */ public long getAvailableSubscribeRateLimit(ConsumerIdentifier consumerIdentifier) { return subscribeRateLimiter.get(consumerIdentifier) - == null ? -1 : subscribeRateLimiter.get(consumerIdentifier).getAvailablePermits(); + == null ? -1 : subscribeRateLimiter.get(consumerIdentifier).getTokens(); } /** @@ -74,8 +66,15 @@ public long getAvailableSubscribeRateLimit(ConsumerIdentifier consumerIdentifier */ public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { addSubscribeLimiterIfAbsent(consumerIdentifier); - return subscribeRateLimiter.get(consumerIdentifier) - == null || subscribeRateLimiter.get(consumerIdentifier).tryAcquire(); + AsyncTokenBucket tokenBucket = subscribeRateLimiter.get(consumerIdentifier); + if (tokenBucket == null) { + return true; + } + if (!tokenBucket.containsTokens(true)) { + return false; + } + tokenBucket.consumeTokens(1); + return true; } /** @@ -86,7 +85,7 @@ public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { */ public boolean subscribeAvailable(ConsumerIdentifier consumerIdentifier) { return (subscribeRateLimiter.get(consumerIdentifier) - == null || subscribeRateLimiter.get(consumerIdentifier).getAvailablePermits() > 0); + == null || subscribeRateLimiter.get(consumerIdentifier).containsTokens()); } /** @@ -98,15 +97,11 @@ private synchronized void addSubscribeLimiterIfAbsent(ConsumerIdentifier consume if (subscribeRateLimiter.get(consumerIdentifier) != null || !isSubscribeRateEnabled(this.subscribeRate)) { return; } - updateSubscribeRate(consumerIdentifier, this.subscribeRate); } private synchronized void removeSubscribeLimiter(ConsumerIdentifier consumerIdentifier) { - if (this.subscribeRateLimiter.get(consumerIdentifier) != null) { - this.subscribeRateLimiter.get(consumerIdentifier).close(); - this.subscribeRateLimiter.remove(consumerIdentifier); - } + this.subscribeRateLimiter.remove(consumerIdentifier); } /** @@ -117,23 +112,13 @@ private synchronized void removeSubscribeLimiter(ConsumerIdentifier consumerIden */ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentifier, SubscribeRate subscribeRate) { long ratePerConsumer = subscribeRate.subscribeThrottlingRatePerConsumer; - long ratePeriod = subscribeRate.ratePeriodInSecond; + long ratePeriodNanos = TimeUnit.SECONDS.toNanos(Math.max(subscribeRate.ratePeriodInSecond, 1)); // update subscribe-rateLimiter if (ratePerConsumer > 0) { - if (this.subscribeRateLimiter.get(consumerIdentifier) == null) { - this.subscribeRateLimiter.put(consumerIdentifier, - RateLimiter.builder() - .scheduledExecutorService(brokerService.pulsar().getExecutor()) - .permits(ratePerConsumer) - .rateTime(ratePeriod) - .timeUnit(TimeUnit.SECONDS) - .build()); - } else { - this.subscribeRateLimiter.get(consumerIdentifier) - .setRate(ratePerConsumer, ratePeriod, TimeUnit.SECONDS, - null); - } + AsyncTokenBucket tokenBucket = + AsyncTokenBucket.builder().rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); + this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); } else { // subscribe-rate should be disable and close removeSubscribeLimiter(consumerIdentifier); @@ -145,7 +130,6 @@ public void onSubscribeRateUpdate(SubscribeRate subscribeRate) { return; } this.subscribeRate = subscribeRate; - stopResetTask(); for (ConsumerIdentifier consumerIdentifier : this.subscribeRateLimiter.keySet()) { if (!isSubscribeRateEnabled(this.subscribeRate)) { removeSubscribeLimiter(consumerIdentifier); @@ -154,7 +138,6 @@ public void onSubscribeRateUpdate(SubscribeRate subscribeRate) { } } if (isSubscribeRateEnabled(this.subscribeRate)) { - this.resetTask = createTask(); log.info("[{}] configured subscribe-dispatch rate at broker {}", this.topicName, subscribeRate); } } @@ -209,32 +192,9 @@ public static boolean isSubscribeRateEnabled(SubscribeRate subscribeRate) { } public void close() { - closeAndClearRateLimiters(); - stopResetTask(); - } - private ScheduledFuture createTask() { - return executorService.scheduleAtFixedRate(catchingAndLoggingThrowables(this::closeAndClearRateLimiters), - this.subscribeRate.ratePeriodInSecond, - this.subscribeRate.ratePeriodInSecond, - TimeUnit.SECONDS); } - private void stopResetTask() { - if (this.resetTask != null) { - this.resetTask.cancel(false); - } - } - - private synchronized void closeAndClearRateLimiters() { - // close rate-limiter - this.subscribeRateLimiter.values().forEach(rateLimiter -> { - if (rateLimiter != null) { - rateLimiter.close(); - } - }); - this.subscribeRateLimiter.clear(); - } public SubscribeRate getSubscribeRate() { return subscribeRate; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index bc2dd58a5812e..978536c5f4e36 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -212,9 +212,12 @@ public TransactionBufferReader newReader(long sequenceId) throws final ConcurrentMap buffers; final Map> txnIndex; + private final Topic topic; + public InMemTransactionBuffer(Topic topic) { this.buffers = new ConcurrentHashMap<>(); this.txnIndex = new HashMap<>(); + this.topic = topic; } @Override @@ -372,7 +375,7 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position) { @Override public PositionImpl getMaxReadPosition() { - return PositionImpl.LATEST; + return (PositionImpl) topic.getLastPosition(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 3c13be220869f..f356921d6988e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -446,8 +447,7 @@ void updateMaxReadPosition(TxnID txnID) { ongoingTxns.remove(txnID); if (!ongoingTxns.isEmpty()) { PositionImpl position = ongoingTxns.get(ongoingTxns.firstKey()); - //max read position is less than first ongoing transaction message position, so entryId -1 - maxReadPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId() - 1); + maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position); } else { maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index 7c74b52951e28..9de0888ae5b0b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; @@ -40,6 +41,11 @@ @Slf4j public class TransactionBufferDisable implements TransactionBuffer { + private final Topic topic; + public TransactionBufferDisable(Topic topic) { + this.topic = topic; + } + @Override public CompletableFuture getTransactionMeta(TxnID txnID) { return CompletableFuture.completedFuture(null); @@ -91,7 +97,7 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position) { @Override public PositionImpl getMaxReadPosition() { - return PositionImpl.LATEST; + return (PositionImpl) topic.getLastPosition(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java index 9aac9ab64d0fd..34ee28693b4fc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; @@ -300,7 +301,7 @@ protected OpRequestSend newObject(Handle handle) { } public CompletableFuture getClientCnxWithLookup(String topic) { - return pulsarClient.getConnection(topic, randomKeyForSelectConnection); + return pulsarClient.getConnection(topic, randomKeyForSelectConnection).thenApply(Pair::getLeft); } public CompletableFuture getClientCnx(String topic) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index 5602f662f5008..e8192cde3fdf3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -1205,7 +1205,7 @@ protected CompletableFuture canUpdateCluster(String tenant, Set ol protected void validateBrokerName(String broker) { String brokerUrl = String.format("http://%s", broker); String brokerUrlTls = String.format("https://%s", broker); - if (!brokerUrl.equals(pulsar().getSafeWebServiceAddress()) + if (!brokerUrl.equals(pulsar().getWebServiceAddress()) && !brokerUrlTls.equals(pulsar().getWebServiceAddressTls())) { String[] parts = broker.split(":"); checkArgument(parts.length == 2, String.format("Invalid broker url %s", broker)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index 5fa64e9f067cc..647c34a94ad81 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -45,6 +45,7 @@ import org.apache.pulsar.client.impl.RawBatchConverter; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,7 +63,7 @@ public class TwoPhaseCompactor extends Compactor { private static final Logger log = LoggerFactory.getLogger(TwoPhaseCompactor.class); private static final int MAX_OUTSTANDING = 500; private final Duration phaseOneLoopReadTimeout; - private final boolean topicCompactionRemainNullKey; + private final boolean topicCompactionRetainNullKey; public TwoPhaseCompactor(ServiceConfiguration conf, PulsarClient pulsar, @@ -70,7 +71,7 @@ public TwoPhaseCompactor(ServiceConfiguration conf, ScheduledExecutorService scheduler) { super(conf, pulsar, bk, scheduler); phaseOneLoopReadTimeout = Duration.ofSeconds(conf.getBrokerServiceCompactionPhaseOneLoopTimeInSeconds()); - topicCompactionRemainNullKey = conf.isTopicCompactionRemainNullKey(); + topicCompactionRetainNullKey = conf.isTopicCompactionRetainNullKey(); } @Override @@ -130,14 +131,17 @@ private void phaseOneLoop(RawReader reader, boolean replaceMessage = false; mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); - if (RawBatchConverter.isReadableBatch(metadata)) { + if (Markers.isServerOnlyMarker(metadata)) { + mxBean.addCompactionRemovedEvent(reader.getTopic()); + deletedMessage = true; + } else if (RawBatchConverter.isReadableBatch(metadata)) { try { int numMessagesInBatch = metadata.getNumMessagesInBatch(); int deleteCnt = 0; for (ImmutableTriple e : extractIdsAndKeysAndSizeFromBatch(m)) { if (e != null) { if (e.getMiddle() == null) { - if (!topicCompactionRemainNullKey) { + if (!topicCompactionRetainNullKey) { // record delete null-key message event deleteCnt++; mxBean.addCompactionRemovedEvent(reader.getTopic()); @@ -174,7 +178,7 @@ private void phaseOneLoop(RawReader reader, latestForKey.remove(keyAndSize.getLeft()); } } else { - if (!topicCompactionRemainNullKey) { + if (!topicCompactionRetainNullKey) { deletedMessage = true; } } @@ -262,10 +266,13 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map MessageId id = m.getMessageId(); Optional messageToAdd = Optional.empty(); mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); - if (RawBatchConverter.isReadableBatch(m)) { + MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); + if (Markers.isServerOnlyMarker(metadata)) { + messageToAdd = Optional.empty(); + } else if (RawBatchConverter.isReadableBatch(metadata)) { try { messageToAdd = rebatchMessage(reader.getTopic(), - m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRemainNullKey); + m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRetainNullKey); } catch (IOException ioe) { log.info("Error decoding batch for message {}. Whole batch will be included in output", id, ioe); @@ -275,7 +282,7 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map Pair keyAndSize = extractKeyAndSize(m); MessageId msg; if (keyAndSize == null) { - messageToAdd = topicCompactionRemainNullKey ? Optional.of(m) : Optional.empty(); + messageToAdd = topicCompactionRetainNullKey ? Optional.of(m) : Optional.empty(); } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null && msg.equals(id)) { // consider message only if present into latestForKey map if (keyAndSize.getRight() <= 0) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java index c9a07dc966de6..12f231a4d2ce3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java @@ -18,14 +18,18 @@ */ package org.apache.pulsar.broker.admin; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.fail; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; import javax.ws.rs.core.Response; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -69,4 +73,38 @@ public void TestDeleteInvalidDynamicConfiguration() { } } } + + @Test + public void testRegisterCustomDynamicConfiguration() throws PulsarAdminException { + String key = "my-broker-config-key-1"; + String invalidValue = "invalid-value"; + + // register + pulsar.getBrokerService().registerCustomDynamicConfiguration(key, value -> !value.equals(invalidValue)); + assertThrows(IllegalArgumentException.class, + () -> pulsar.getBrokerService().registerCustomDynamicConfiguration(key, null)); + Map allDynamicConfigurations = admin.brokers().getAllDynamicConfigurations(); + assertThat(allDynamicConfigurations).doesNotContainKey(key); + + // update with listener + AtomicReference changeValue = new AtomicReference<>(null); + pulsar.getBrokerService().registerConfigurationListener(key, changeValue::set); + String newValue = "my-broker-config-value-1"; + admin.brokers().updateDynamicConfiguration(key, newValue); + allDynamicConfigurations = admin.brokers().getAllDynamicConfigurations(); + assertThat(allDynamicConfigurations.get(key)).isEqualTo(newValue); + + Awaitility.await().untilAsserted(() -> { + assertThat(changeValue.get()).isEqualTo(newValue); + }); + + // update with invalid value + assertThrows(PulsarAdminException.PreconditionFailedException.class, + () -> admin.brokers().updateDynamicConfiguration(key, invalidValue)); + + // delete + admin.brokers().deleteDynamicConfiguration(key); + allDynamicConfigurations = admin.brokers().getAllDynamicConfigurations(); + assertThat(allDynamicConfigurations).doesNotContainKey(key); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index f66761ff95aa5..7939b19283946 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -429,6 +429,7 @@ public void testTerminate() { persistentTopics.createNonPartitionedTopic(response, testTenant, testNamespace, testLocalTopicName, true, null); // 2) Create a subscription + response = mock(AsyncResponse.class); persistentTopics.createSubscription(response, testTenant, testNamespace, testLocalTopicName, "test", true, new ResetCursorData(MessageId.earliest), false); ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); @@ -539,12 +540,13 @@ public void testCreateNonPartitionedTopic() { @Test public void testCreatePartitionedTopic() { - AsyncResponse response = mock(AsyncResponse.class); - ArgumentCaptor responseCaptor = - ArgumentCaptor.forClass(PartitionedTopicMetadata.class); final String topicName = "standard-partitioned-topic-a"; - persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, topicName, 2, true); + persistentTopics.createPartitionedTopic(mock(AsyncResponse.class), testTenant, testNamespace, topicName, 2, + true); Awaitility.await().untilAsserted(() -> { + ArgumentCaptor responseCaptor = + ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + AsyncResponse response = mock(AsyncResponse.class); persistentTopics.getPartitionedMetadata(response, testTenant, testNamespace, topicName, true, false); verify(response, timeout(5000).atLeast(1)).resume(responseCaptor.capture()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java index c9138beee52d1..9cd1cf214f67e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java @@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; @@ -40,6 +39,7 @@ import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.LookupTopicResult; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; @@ -135,10 +135,10 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() ((PulsarClientImpl) pulsarClient).setLookup(mockLookup); when(mockLookup.getPartitionedTopicMetadata(any())).thenAnswer( i -> CompletableFuture.completedFuture(new PartitionedTopicMetadata(0))); - when(mockLookup.getBroker(any())).thenAnswer(i -> { + when(mockLookup.getBroker(any())).thenAnswer(ignored -> { InetSocketAddress brokerAddress = new InetSocketAddress(pulsar.getAdvertisedAddress(), pulsar.getBrokerListenPort().get()); - return CompletableFuture.completedFuture(Pair.of(brokerAddress, brokerAddress)); + return CompletableFuture.completedFuture(new LookupTopicResult(brokerAddress, brokerAddress, false)); }); final String topicPoliciesServiceInitException = "Topic creation encountered an exception by initialize topic policies service"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 25ca3bf1444d2..036d4354a5f1b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -2023,16 +2023,12 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception { final String topicName = "persistent://" + myNamespace + "/test-" + UUID.randomUUID(); pulsarClient.newProducer().topic(topicName).create().close(); - Field publishMaxMessageRate = PublishRateLimiterImpl.class.getDeclaredField("publishMaxMessageRate"); - publishMaxMessageRate.setAccessible(true); - Field publishMaxByteRate = PublishRateLimiterImpl.class.getDeclaredField("publishMaxByteRate"); - publishMaxByteRate.setAccessible(true); //1 use broker-level policy by default PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); PublishRateLimiterImpl publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - Assert.assertEquals(publishMaxMessageRate.get(publishRateLimiter), 5); - Assert.assertEquals(publishMaxByteRate.get(publishRateLimiter), 50L); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 5); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 50L); //2 set namespace-level policy PublishRate publishMsgRate = new PublishRate(10, 100L); @@ -2041,12 +2037,12 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception { Awaitility.await() .until(() -> { PublishRateLimiterImpl limiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - return (int)publishMaxMessageRate.get(limiter) == 10; + return (int)limiter.getTokenBucketOnMessage().getRate() == 10; }); publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - Assert.assertEquals(publishMaxMessageRate.get(publishRateLimiter), 10); - Assert.assertEquals(publishMaxByteRate.get(publishRateLimiter), 100L); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 10); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 100L); //3 set topic-level policy, namespace-level policy should be overwritten PublishRate publishMsgRate2 = new PublishRate(11, 101L); @@ -2056,8 +2052,8 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception { .until(() -> admin.topicPolicies().getPublishRate(topicName) != null); publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - Assert.assertEquals(publishMaxMessageRate.get(publishRateLimiter), 11); - Assert.assertEquals(publishMaxByteRate.get(publishRateLimiter), 101L); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 11); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 101L); //4 remove topic-level policy, namespace-level policy will take effect admin.topicPolicies().removePublishRate(topicName); @@ -2066,8 +2062,8 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception { .until(() -> admin.topicPolicies().getPublishRate(topicName) == null); publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - Assert.assertEquals(publishMaxMessageRate.get(publishRateLimiter), 10); - Assert.assertEquals(publishMaxByteRate.get(publishRateLimiter), 100L); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 10); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 100L); //5 remove namespace-level policy, broker-level policy will take effect admin.namespaces().removePublishRate(myNamespace); @@ -2075,12 +2071,12 @@ public void testPublishRateInDifferentLevelPolicy() throws Exception { Awaitility.await() .until(() -> { PublishRateLimiterImpl limiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - return (int)publishMaxMessageRate.get(limiter) == 5; + return (int)limiter.getTokenBucketOnMessage().getRate() == 5; }); publishRateLimiter = (PublishRateLimiterImpl) topic.getTopicPublishRateLimiter(); - Assert.assertEquals(publishMaxMessageRate.get(publishRateLimiter), 5); - Assert.assertEquals(publishMaxByteRate.get(publishRateLimiter), 50L); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnMessage().getRate(), 5); + Assert.assertEquals(publishRateLimiter.getTokenBucketOnByte().getRate(), 50L); } @Test(timeOut = 20000) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BundlesQuotasTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BundlesQuotasTest.java index d78e8c0914c7e..079ce25318a6a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BundlesQuotasTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BundlesQuotasTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.Range; import com.google.common.hash.Hashing; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.resources.LoadBalanceResources; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceName; @@ -41,14 +43,24 @@ public class BundlesQuotasTest { private MetadataStore store; private NamespaceBundleFactory bundleFactory; + private PulsarService pulsar; @BeforeMethod public void setup() throws Exception { store = MetadataStoreFactory.create("memory:local", MetadataStoreConfig.builder().build()); + LoadBalanceResources.QuotaResources quotaResources = new LoadBalanceResources.QuotaResources(store, 30000); - PulsarService pulsar = mock(PulsarService.class); + pulsar = mock(PulsarService.class); when(pulsar.getLocalMetadataStore()).thenReturn(mock(MetadataStoreExtended.class)); when(pulsar.getConfigurationMetadataStore()).thenReturn(mock(MetadataStoreExtended.class)); + + LoadBalanceResources loadBalanceResources = mock(LoadBalanceResources.class); + when(loadBalanceResources.getQuotaResources()).thenReturn(quotaResources); + + PulsarResources pulsarResources = mock(PulsarResources.class); + when(pulsarResources.getLoadBalanceResources()).thenReturn(loadBalanceResources); + + when(pulsar.getPulsarResources()).thenReturn(pulsarResources); bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); } @@ -59,7 +71,7 @@ public void teardown() throws Exception { @Test public void testGetSetDefaultQuota() throws Exception { - BundlesQuotas bundlesQuotas = new BundlesQuotas(store); + BundlesQuotas bundlesQuotas = new BundlesQuotas(pulsar); ResourceQuota quota2 = new ResourceQuota(); quota2.setMsgRateIn(10); quota2.setMsgRateOut(20); @@ -75,7 +87,7 @@ public void testGetSetDefaultQuota() throws Exception { @Test public void testGetSetBundleQuota() throws Exception { - BundlesQuotas bundlesQuotas = new BundlesQuotas(store); + BundlesQuotas bundlesQuotas = new BundlesQuotas(pulsar); NamespaceBundle testBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-2"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index 7f2767b2e771a..43706129fbe15 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -24,7 +24,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import com.fasterxml.jackson.databind.ObjectMapper; @@ -56,12 +56,16 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit; import org.apache.pulsar.client.admin.BrokerStats; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.ResourceQuota; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.api.MetadataStoreException.BadVersionException; @@ -72,6 +76,7 @@ import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -93,8 +98,14 @@ public class SimpleLoadManagerImplTest { BrokerStats brokerStatsClient2; String primaryHost; + + String primaryTlsHost; + String secondaryHost; + private String defaultNamespace; + private String defaultTenant; + ExecutorService executor = new ThreadPoolExecutor(5, 20, 30, TimeUnit.SECONDS, new LinkedBlockingQueue<>()); @BeforeMethod @@ -108,6 +119,7 @@ void setup() throws Exception { ServiceConfiguration config1 = new ServiceConfiguration(); config1.setClusterName("use"); config1.setWebServicePort(Optional.of(0)); + config1.setWebServicePortTls(Optional.of(0)); config1.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config1.setBrokerShutdownTimeoutMs(0L); config1.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); @@ -122,11 +134,13 @@ void setup() throws Exception { admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); brokerStatsClient1 = admin1.brokerStats(); primaryHost = pulsar1.getWebServiceAddress(); + primaryTlsHost = pulsar1.getWebServiceAddressTls(); // Start broker 2 ServiceConfiguration config2 = new ServiceConfiguration(); config2.setClusterName("use"); config2.setWebServicePort(Optional.of(0)); + config2.setWebServicePortTls(Optional.of(0)); config2.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config2.setBrokerShutdownTimeoutMs(0L); config2.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); @@ -143,6 +157,8 @@ void setup() throws Exception { brokerStatsClient2 = admin2.brokerStats(); secondaryHost = pulsar2.getWebServiceAddress(); Thread.sleep(100); + + setupClusters(); } @AfterMethod(alwaysRun = true) @@ -256,10 +272,9 @@ public void testPrimary() throws Exception { sortedRankingsInstance.get().put(lr.getRank(rd), rus); setObjectField(SimpleLoadManagerImpl.class, loadManager, "sortedRankings", sortedRankingsInstance); - ResourceUnit found = loadManager - .getLeastLoaded(NamespaceName.get("pulsar/use/primary-ns.10")).get(); + final Optional leastLoaded = loadManager.getLeastLoaded(NamespaceName.get("pulsar/use/primary-ns.10")); // broker is not active so found should be null - assertNotEquals(found, null, "did not find a broker when expected one to be found"); + assertFalse(leastLoaded.isPresent()); } @@ -399,7 +414,7 @@ public void testEvenBundleDistribution() throws Exception { final SimpleLoadManagerImpl loadManager = (SimpleLoadManagerImpl) pulsar1.getLoadManager().get(); for (final NamespaceBundle bundle : bundles) { - if (loadManager.getLeastLoaded(bundle).get().getResourceId().equals(primaryHost)) { + if (loadManager.getLeastLoaded(bundle).get().getResourceId().equals(getAddress(primaryTlsHost))) { ++numAssignedToPrimary; } else { ++numAssignedToSecondary; @@ -411,6 +426,10 @@ public void testEvenBundleDistribution() throws Exception { } } + private static String getAddress(String url) { + return url.replaceAll("https", "http"); + } + @Test public void testNamespaceBundleStats() { NamespaceBundleStats nsb1 = new NamespaceBundleStats(); @@ -479,4 +498,33 @@ public void testUsage() { assertEquals(usage.getBandwidthIn().usage, usageLimit); } + @Test + public void testGetWebSerUrl() throws PulsarAdminException { + String webServiceUrl = admin1.brokerStats().getLoadReport().getWebServiceUrl(); + Assert.assertEquals(webServiceUrl, pulsar1.getWebServiceAddress()); + + String webServiceUrl2 = admin2.brokerStats().getLoadReport().getWebServiceUrl(); + Assert.assertEquals(webServiceUrl2, pulsar2.getWebServiceAddress()); + } + + @Test + public void testRedirectOwner() throws PulsarAdminException { + final String topicName = "persistent://" + defaultNamespace + "/" + "test-topic"; + admin1.topics().createNonPartitionedTopic(topicName); + TopicStats stats = admin1.topics().getStats(topicName); + Assert.assertNotNull(stats); + + TopicStats stats2 = admin2.topics().getStats(topicName); + Assert.assertNotNull(stats2); + } + + private void setupClusters() throws PulsarAdminException { + admin1.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar1.getWebServiceAddress()).build()); + TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("use")); + defaultTenant = "prop-xyz"; + admin1.tenants().createTenant(defaultTenant, tenantInfo); + defaultNamespace = defaultTenant + "/ns1"; + admin1.namespaces().createNamespace(defaultNamespace, Set.of("use")); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index fb49071fa4701..bb7416ddc4103 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -38,11 +38,10 @@ import static org.apache.pulsar.broker.namespace.NamespaceService.getHeartbeatNamespace; import static org.apache.pulsar.broker.namespace.NamespaceService.getHeartbeatNamespaceV2; import static org.apache.pulsar.broker.namespace.NamespaceService.getSLAMonitorNamespace; -import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -55,19 +54,27 @@ import static org.testng.Assert.fail; import com.google.common.collect.Sets; import java.net.URL; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.reflect.FieldUtils; @@ -100,13 +107,20 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.policies.data.BrokerAssignment; @@ -120,7 +134,6 @@ import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; import org.awaitility.Awaitility; -import org.mockito.MockedStatic; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -154,53 +167,47 @@ public class ExtensibleLoadManagerImplTest extends MockedPulsarServiceBaseTest { @BeforeClass @Override public void setup() throws Exception { - try (MockedStatic channelMockedStatic = - mockStatic(ServiceUnitStateChannelImpl.class)) { - channelMockedStatic.when(() -> ServiceUnitStateChannelImpl.newInstance(isA(PulsarService.class))) - .thenAnswer(invocation -> { - PulsarService pulsarService = invocation.getArgument(0); - // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid - // stuck when doing unload. - return new ServiceUnitStateChannelImpl(pulsarService, 5 * 1000, 1); - }); - conf.setForceDeleteNamespaceAllowed(true); - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); - conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - conf.setLoadBalancerSheddingEnabled(false); - conf.setLoadBalancerDebugModeEnabled(true); - conf.setTopicLevelPoliciesEnabled(true); - super.internalSetup(conf); - pulsar1 = pulsar; - ServiceConfiguration defaultConf = getDefaultConf(); - defaultConf.setAllowAutoTopicCreation(true); - defaultConf.setForceDeleteNamespaceAllowed(true); - defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - defaultConf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - defaultConf.setLoadBalancerSheddingEnabled(false); - defaultConf.setTopicLevelPoliciesEnabled(true); - additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); - pulsar2 = additionalPulsarTestContext.getPulsarService(); - - setPrimaryLoadManager(); - - setSecondaryLoadManager(); - - admin.clusters().createCluster(this.conf.getClusterName(), - ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); - admin.tenants().createTenant("public", - new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), - Sets.newHashSet(this.conf.getClusterName()))); - admin.namespaces().createNamespace("public/default"); - admin.namespaces().setNamespaceReplicationClusters("public/default", - Sets.newHashSet(this.conf.getClusterName())); - - admin.namespaces().createNamespace(defaultTestNamespace, 128); - admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, - Sets.newHashSet(this.conf.getClusterName())); - lookupService = (LookupService) FieldUtils.readDeclaredField(pulsarClient, "lookup", true); - } + // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid + // stuck when doing unload. + conf.setLoadBalancerInFlightServiceUnitStateWaitingTimeInMillis(5 * 1000); + conf.setLoadBalancerServiceUnitStateMonitorIntervalInSeconds(1); + conf.setForceDeleteNamespaceAllowed(true); + conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + conf.setAllowAutoTopicCreation(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadBalancerSheddingEnabled(false); + conf.setLoadBalancerDebugModeEnabled(true); + conf.setTopicLevelPoliciesEnabled(true); + super.internalSetup(conf); + pulsar1 = pulsar; + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + defaultConf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + defaultConf.setTopicLevelPoliciesEnabled(true); + additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); + pulsar2 = additionalPulsarTestContext.getPulsarService(); + + setPrimaryLoadManager(); + + setSecondaryLoadManager(); + + admin.clusters().createCluster(this.conf.getClusterName(), + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), + Sets.newHashSet(this.conf.getClusterName()))); + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", + Sets.newHashSet(this.conf.getClusterName())); + + admin.namespaces().createNamespace(defaultTestNamespace, 128); + admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, + Sets.newHashSet(this.conf.getClusterName())); + lookupService = (LookupService) FieldUtils.readDeclaredField(pulsarClient, "lookup", true); } @Override @@ -419,110 +426,240 @@ public boolean test(NamespaceBundle namespaceBundle) { assertTrue(ex.getMessage().contains("cannot be transfer to same broker")); } } - @DataProvider(name = "isPersistentTopicTest") - public Object[][] isPersistentTopicTest() { - return new Object[][] { { true }, { false }}; + + @DataProvider(name = "isPersistentTopicSubscriptionTypeTest") + public Object[][] isPersistentTopicSubscriptionTypeTest() { + return new Object[][]{ + {TopicDomain.persistent, SubscriptionType.Exclusive}, + {TopicDomain.persistent, SubscriptionType.Shared}, + {TopicDomain.persistent, SubscriptionType.Failover}, + {TopicDomain.persistent, SubscriptionType.Key_Shared}, + {TopicDomain.non_persistent, SubscriptionType.Exclusive}, + {TopicDomain.non_persistent, SubscriptionType.Shared}, + {TopicDomain.non_persistent, SubscriptionType.Failover}, + {TopicDomain.non_persistent, SubscriptionType.Key_Shared}, + }; } - @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") - public void testTransferClientReconnectionWithoutLookup(boolean isPersistentTopicTest) throws Exception { - String topicType = isPersistentTopicTest? "persistent" : "non-persistent"; - String topic = topicType + "://" + defaultTestNamespace + "/test-transfer-client-reconnect"; - TopicName topicName = TopicName.get(topic); - AtomicInteger lookupCount = new AtomicInteger(); - var lookup = spyLookupService(lookupCount, topicName); - var producer = pulsarClient.newProducer().topic(topic).create(); - int lookupCountBeforeUnload = lookupCount.get(); + @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") + public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) + throws Exception { + var id = String.format("test-tx-client-reconnect-%s-%s", subscriptionType, UUID.randomUUID()); + var topic = String.format("%s://%s/%s", topicDomain.toString(), defaultTestNamespace, id); + var topicName = TopicName.get(topic); + var timeoutMs = 30_000; - NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); - String broker = admin.lookups().lookupTopic(topic); - String dstBrokerUrl = pulsar1.getLookupServiceAddress(); - String dstBrokerServiceUrl; - if (broker.equals(pulsar1.getBrokerServiceUrl())) { - dstBrokerUrl = pulsar2.getLookupServiceAddress(); - dstBrokerServiceUrl = pulsar2.getBrokerServiceUrl(); - } else { - dstBrokerServiceUrl = pulsar1.getBrokerServiceUrl(); - } - checkOwnershipState(broker, bundle); + var clients = new ArrayList(); + var consumers = new ArrayList>(); + try { + var lookups = new ArrayList(); + + @Cleanup + var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + lookups.add(spyLookupService(pulsarClient)); + + var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; + + for (int i = 0; i < consumerCount; i++) { + var client = newPulsarClient(lookupUrl.toString(), 0); + clients.add(client); + var consumer = client.newConsumer(Schema.STRING). + subscriptionName(id). + subscriptionType(subscriptionType). + subscriptionInitialPosition(SubscriptionInitialPosition.Earliest). + ackTimeout(1000, TimeUnit.MILLISECONDS). + topic(topic). + subscribe(); + consumers.add(consumer); + lookups.add(spyLookupService(client)); + } + + Awaitility.await() + .until(() -> producer.isConnected() && consumers.stream().allMatch(Consumer::isConnected)); + + NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); + String broker = admin.lookups().lookupTopic(topic); + final String dstBrokerUrl; + final String dstBrokerServiceUrl; + if (broker.equals(pulsar1.getBrokerServiceUrl())) { + dstBrokerUrl = pulsar2.getLookupServiceAddress(); + dstBrokerServiceUrl = pulsar2.getBrokerServiceUrl(); + } else { + dstBrokerUrl = pulsar1.getLookupServiceAddress(); + dstBrokerServiceUrl = pulsar1.getBrokerServiceUrl(); + } + checkOwnershipState(broker, bundle); + + var messageCountBeforeUnloading = 100; + var messageCountAfterUnloading = 100; + var messageCount = messageCountBeforeUnloading + messageCountAfterUnloading; + + var semMessagesReadyToSend = new Semaphore(0); + var cdlStart = new CountDownLatch(1); - final String finalDstBrokerUrl = dstBrokerUrl; - CompletableFuture.runAsync(() -> { + @Cleanup(value = "shutdown") + var executor = Executors.newFixedThreadPool(1 /* bundle unload */ + 1 /* producer */ + consumers.size()); + + var futures = new ArrayList>(); + futures.add(CompletableFuture.runAsync(() -> { try { - admin.namespaces().unloadNamespaceBundle( - defaultTestNamespace, bundle.getBundleRange(), finalDstBrokerUrl); - } catch (PulsarAdminException e) { - throw new RuntimeException(e); + cdlStart.await(); + semMessagesReadyToSend.release(messageCountBeforeUnloading); + admin.namespaces() + .unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), dstBrokerUrl); + semMessagesReadyToSend.release(messageCountAfterUnloading); + } catch (InterruptedException | PulsarAdminException e) { + fail(); } - } - ); + }, executor)); - Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { - try { - producer.send("hi".getBytes()); - String newOwner = admin.lookups().lookupTopic(topic); - assertEquals(dstBrokerServiceUrl, newOwner); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } catch (PulsarAdminException e) { - throw new RuntimeException(e); + var pendingMessages = Collections.synchronizedSet(new HashSet<>(messageCount)); + var producerFuture = CompletableFuture.runAsync(() -> { + try { + cdlStart.await(); + for (int i = 0; i < messageCount; i++) { + semMessagesReadyToSend.acquire(); + String message = String.format("message-%d", i); + if (topicDomain == TopicDomain.persistent) { + // Only verify receipt of persistent topic messages. + pendingMessages.add(message); + } + producer.send(message); + } + } catch (PulsarClientException | InterruptedException e) { + fail(); + } + }, executor); + futures.add(producerFuture); + + consumers.stream().map(consumer -> CompletableFuture.runAsync(() -> { + try { + cdlStart.await(); + } catch (InterruptedException e) { + fail(); + } + while (!producerFuture.isDone() || !pendingMessages.isEmpty()) { + try { + var message = consumer.receive(1500, TimeUnit.MILLISECONDS); + if (message != null) { + consumer.acknowledge(message); + pendingMessages.remove(message.getValue()); + } + } catch (PulsarClientException e) { + // Retry read + } + } + }, executor)).forEach(futures::add); + + var asyncTasks = FutureUtil.waitForAllAndSupportCancel(futures).orTimeout(timeoutMs, TimeUnit.MILLISECONDS); + + cdlStart.countDown(); + Awaitility.await().atMost(timeoutMs, TimeUnit.MILLISECONDS).ignoreExceptions().until( + () -> dstBrokerServiceUrl.equals(admin.lookups().lookupTopic(topic))); + + asyncTasks.get(); + + assertTrue(futures.stream().allMatch(CompletableFuture::isDone)); + assertTrue(futures.stream().noneMatch(CompletableFuture::isCompletedExceptionally)); + assertTrue(pendingMessages.isEmpty()); + + assertTrue(producer.isConnected()); + assertTrue(consumers.stream().allMatch(Consumer::isConnected)); + + for (LookupService lookupService : lookups) { + verify(lookupService, never()).getBroker(topicName); } - }); - assertTrue(producer.isConnected()); - verify(lookup, times(lookupCountBeforeUnload)).getBroker(topicName); - producer.close(); + } finally { + for (var consumer: consumers) { + consumer.close(); + } + for (var client: clients) { + client.close(); + } + } } + @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") + public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, + SubscriptionType subscriptionType) throws Exception { + var id = String.format("test-unload-%s-client-reconnect-%s-%s", + topicDomain, subscriptionType, UUID.randomUUID()); + var topic = String.format("%s://%s/%s", topicDomain, defaultTestNamespace, id); + var topicName = TopicName.get(topic); + var consumers = new ArrayList>(); + try { + @Cleanup + var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); - @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") - public void testUnloadClientReconnectionWithLookup(boolean isPersistentTopicTest) throws Exception { - String topicType = isPersistentTopicTest? "persistent" : "non-persistent"; - String topic = topicType + "://" + defaultTestNamespace + "/test-unload-client-reconnect-" - + isPersistentTopicTest; - TopicName topicName = TopicName.get(topic); + var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; + for (int i = 0; i < consumerCount; i++) { + consumers.add(pulsarClient.newConsumer(Schema.STRING). + subscriptionName(id).subscriptionType(subscriptionType).topic(topic).subscribe()); + } + Awaitility.await() + .until(() -> producer.isConnected() && consumers.stream().allMatch(Consumer::isConnected)); - AtomicInteger lookupCount = new AtomicInteger(); - var lookup = spyLookupService(lookupCount, topicName); + var lookup = spyLookupService(pulsarClient); - var producer = pulsarClient.newProducer().topic(topic).create(); - int lookupCountBeforeUnload = lookupCount.get(); + final CountDownLatch cdl = new CountDownLatch(3); - NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); - CompletableFuture.runAsync(() -> { - try { - admin.namespaces().unloadNamespaceBundle( - defaultTestNamespace, bundle.getBundleRange()); - } catch (PulsarAdminException e) { - throw new RuntimeException(e); + NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); + CompletableFuture unloadNamespaceBundle = CompletableFuture.runAsync(() -> { + try { + cdl.await(); + admin.namespaces().unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange()); + } catch (InterruptedException | PulsarAdminException e) { + fail(); + } + }); + + MutableInt sendCount = new MutableInt(); + Awaitility.await().atMost(20, TimeUnit.SECONDS).ignoreExceptions().until(() -> { + var message = String.format("message-%d", sendCount.getValue()); + + boolean messageSent = false; + while (true) { + var recvFutures = consumers.stream(). + map(consumer -> consumer.receiveAsync().orTimeout(1000, TimeUnit.MILLISECONDS)). + collect(Collectors.toList()); + + if (!messageSent) { + producer.send(message); + messageSent = true; + } + + if (topicDomain == TopicDomain.non_persistent) { + // No need to wait for message receipt, we're only trying to stress the consumer lookup pathway. + break; + } + var msg = (Message) FutureUtil.waitForAny(recvFutures, __ -> true).get().get(); + if (Objects.equals(msg.getValue(), message)) { + break; } } - ); - MutableInt sendCount = new MutableInt(); - Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { - try { - producer.send("hi".getBytes()); - assertEquals(sendCount.incrementAndGet(), 10); - } catch (PulsarClientException e) { - throw new RuntimeException(e); + + cdl.countDown(); + return sendCount.incrementAndGet() == 10; + }); + + assertTrue(producer.isConnected()); + assertTrue(consumers.stream().allMatch(Consumer::isConnected)); + assertTrue(unloadNamespaceBundle.isDone()); + verify(lookup, times(1 + consumerCount)).getBroker(topicName); + } finally { + for (var consumer : consumers) { + consumer.close(); } - }); - assertTrue(producer.isConnected()); - verify(lookup, times(lookupCountBeforeUnload + 1)).getBroker(topicName); - producer.close(); + } } - private LookupService spyLookupService(AtomicInteger lookupCount, TopicName topicName) - throws IllegalAccessException { - var lookup = spy(lookupService); - FieldUtils.writeDeclaredField(pulsarClient, "lookup", lookup, true); - doAnswer(invocationOnMock -> { - lookupCount.incrementAndGet(); - return invocationOnMock.callRealMethod(); - }).when(lookup).getBroker(topicName); + private LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { + LookupService svc = (LookupService) FieldUtils.readDeclaredField(client, "lookup", true); + var lookup = spy(svc); + FieldUtils.writeDeclaredField(client, "lookup", lookup, true); return lookup; } - private void checkOwnershipState(String broker, NamespaceBundle bundle) throws ExecutionException, InterruptedException { var targetLoadManager = secondaryLoadManager; @@ -568,15 +705,21 @@ public boolean test(NamespaceBundle namespaceBundle) { admin.namespaces().splitNamespaceBundle(namespace, firstBundle, true, null); - BundlesData bundlesData = admin.namespaces().getBundles(namespace); - assertEquals(bundlesData.getNumBundles(), numBundles + 1); - String lowBundle = String.format("0x%08x", bundleRanges.get(0)); - String midBundle = String.format("0x%08x", mid); - String highBundle = String.format("0x%08x", bundleRanges.get(1)); - assertTrue(bundlesData.getBoundaries().contains(lowBundle)); - assertTrue(bundlesData.getBoundaries().contains(midBundle)); - assertTrue(bundlesData.getBoundaries().contains(highBundle)); - assertEquals(splitCount.get(), 1); + + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + BundlesData bundlesData = admin.namespaces().getBundles(namespace); + assertEquals(bundlesData.getNumBundles(), numBundles + 1); + String lowBundle = String.format("0x%08x", bundleRanges.get(0)); + String midBundle = String.format("0x%08x", mid); + String highBundle = String.format("0x%08x", bundleRanges.get(1)); + assertTrue(bundlesData.getBoundaries().contains(lowBundle)); + assertTrue(bundlesData.getBoundaries().contains(midBundle)); + assertTrue(bundlesData.getBoundaries().contains(highBundle)); + assertEquals(splitCount.get(), 1); + }); + // Test split bundle with invalid bundle range. try { @@ -619,15 +762,24 @@ public void testDeleteNamespaceBundle() throws Exception { final String namespace = "public/testDeleteNamespaceBundle"; admin.namespaces().createNamespace(namespace, 3); TopicName topicName = TopicName.get(namespace + "/test-delete-namespace-bundle"); - NamespaceBundle bundle = getBundleAsync(pulsar1, topicName).get(); - String broker = admin.lookups().lookupTopic(topicName.toString()); - log.info("Assign the bundle {} to {}", bundle, broker); - checkOwnershipState(broker, bundle); - admin.namespaces().deleteNamespaceBundle(topicName.getNamespace(), bundle.getBundleRange()); - assertFalse(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); + Awaitility.await() + .atMost(30, TimeUnit.SECONDS) + .ignoreExceptions() + .untilAsserted(() -> { + NamespaceBundle bundle = getBundleAsync(pulsar1, topicName).get(); + String broker = admin.lookups().lookupTopic(topicName.toString()); + log.info("Assign the bundle {} to {}", bundle, broker); + checkOwnershipState(broker, bundle); + admin.namespaces().deleteNamespaceBundle(topicName.getNamespace(), bundle.getBundleRange(), true); + // this could fail if the system topic lookup asynchronously happens before this. + // we will retry if it fails. + assertFalse(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); + }); + + admin.namespaces().deleteNamespace(namespace, true); } @Test(timeOut = 30 * 1000) @@ -704,15 +856,6 @@ public CompletableFuture> filterAsync(Map channelMockedStatic = - mockStatic(ServiceUnitStateChannelImpl.class)) { - channelMockedStatic.when(() -> ServiceUnitStateChannelImpl.newInstance(isA(PulsarService.class))) - .thenAnswer(invocation -> { - PulsarService pulsarService = invocation.getArgument(0); - // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid - // stuck when doing unload. - return new ServiceUnitStateChannelImpl(pulsarService, 5 * 1000, 1); - }); // Test rollback to modular load manager. ServiceConfiguration defaultConf = getDefaultConf(); defaultConf.setAllowAutoTopicCreation(true); @@ -833,8 +976,6 @@ public void testDeployAndRollbackLoadManager() throws Exception { } } } - } - } private void assertLookupHeartbeatOwner(PulsarService pulsar, @@ -958,12 +1099,12 @@ public void testRoleChange() throws Exception { FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondarySpy, true); if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - primaryLoadManager.playFollower(); - primaryLoadManager.playFollower(); + primaryLoadManager.playFollower(); // close 3 times + primaryLoadManager.playFollower(); // close 1 time secondaryLoadManager.playLeader(); secondaryLoadManager.playLeader(); - primaryLoadManager.playLeader(); - primaryLoadManager.playLeader(); + primaryLoadManager.playLeader(); // close 3 times and open 3 times + primaryLoadManager.playLeader(); // close 1 time and open 1 time, secondaryLoadManager.playFollower(); secondaryLoadManager.playFollower(); } else { @@ -978,10 +1119,10 @@ public void testRoleChange() throws Exception { } - verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); - verify(topBundlesLoadDataStorePrimarySpy, times(3)).closeTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(3)).startTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(4)).startTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(8)).closeTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(4)).startTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(8)).closeTableView(); FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStorePrimary, true); FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondary, true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index df1bfd12d3eaf..f99a167ff4883 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -622,12 +622,12 @@ public void splitAndRetryTest() throws Exception { FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 1, true); + "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 1 , true); FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 1, true); + "stateTombstoneDelayTimeInMillis", 1, true); ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( List.of(lookupServiceAddress1, lookupServiceAddress2)); @@ -639,7 +639,7 @@ public void splitAndRetryTest() throws Exception { var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; validateMonitorCounters(leader, 0, - 3, + 1, 0, 0, 0, @@ -654,12 +654,12 @@ public void splitAndRetryTest() throws Exception { FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + "stateTombstoneDelayTimeInMillis", 300 * 1000, true); FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + "stateTombstoneDelayTimeInMillis", 300 * 1000, true); } @Test(priority = 7) @@ -1066,12 +1066,12 @@ public void unloadTest() FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 1, true); + "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 1 , true); FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 1, true); + "stateTombstoneDelayTimeInMillis", 1, true); ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( List.of(lookupServiceAddress1, lookupServiceAddress2)); @@ -1094,12 +1094,12 @@ public void unloadTest() FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 30 * 1000, true); + "stateTombstoneDelayTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 300 * 1000, true); FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + "stateTombstoneDelayTimeInMillis", 300 * 1000, true); } @Test(priority = 13) @@ -1396,9 +1396,9 @@ public void splitAndRetryFailureTest() throws Exception { // try the monitor and check the monitor moves `Deleted` -> `Init` FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 1, true); + "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 1, true); + "stateTombstoneDelayTimeInMillis", 1, true); ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( List.of(lookupServiceAddress1, lookupServiceAddress2)); @@ -1409,7 +1409,7 @@ public void splitAndRetryFailureTest() throws Exception { validateMonitorCounters(leader, 0, - 3, + 1, 1, 0, 0, @@ -1422,12 +1422,12 @@ public void splitAndRetryFailureTest() throws Exception { FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + "stateTombstoneDelayTimeInMillis", 300 * 1000, true); FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + "stateTombstoneDelayTimeInMillis", 300 * 1000, true); } @Test(priority = 17) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java index 68bd7b29094cd..a120ef473e9a5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java @@ -90,10 +90,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; configuration.setPreferLaterVersions(true); doReturn(configuration).when(mockContext).brokerConfiguration(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java index 26d95a0158d52..4eec612477758 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java @@ -383,10 +383,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; var topBundleLoadDataStore = new LoadDataStore() { @@ -436,10 +451,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; BrokerRegistry brokerRegistry = mock(BrokerRegistry.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java index 184c337a47c80..7431b9815f93f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java @@ -75,6 +75,7 @@ public void testPushGetAndRemove() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar.getClient(), topic, MyClass.class); + loadDataStore.startProducer(); loadDataStore.startTableView(); MyClass myClass1 = new MyClass("1", 1); loadDataStore.pushAsync("key1", myClass1).get(); @@ -108,6 +109,7 @@ public void testForEach() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + loadDataStore.startProducer(); loadDataStore.startTableView(); Map map = new HashMap<>(); @@ -132,6 +134,7 @@ public void testTableViewRestart() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + loadDataStore.startProducer(); loadDataStore.startTableView(); loadDataStore.pushAsync("1", 1).get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java index 0eea1d87513bf..b1e09bf2f3afb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java @@ -252,10 +252,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; doReturn(conf).when(ctx).brokerConfiguration(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java index 6a6065bc289f6..7004eae29b5ac 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java @@ -278,4 +278,36 @@ public void testDataPojo() { assertEquals(data2.getRedirectLookupAddress(), url); } + @Test + public void topicNotFound() throws Exception { + MockTopicLookup destLookup = spy(MockTopicLookup.class); + doReturn(false).when(destLookup).isRequestHttps(); + BrokerService brokerService = pulsar.getBrokerService(); + doReturn(new Semaphore(1000,true)).when(brokerService).getLookupRequestSemaphore(); + destLookup.setPulsar(pulsar); + doReturn("null").when(destLookup).clientAppId(); + Field uriField = PulsarWebResource.class.getDeclaredField("uri"); + uriField.setAccessible(true); + UriInfo uriInfo = mock(UriInfo.class); + uriField.set(destLookup, uriInfo); + URI uri = URI.create("http://localhost:8080/lookup/v2/destination/topic/myprop/usc/ns2/topic1"); + doReturn(uri).when(uriInfo).getRequestUri(); + config.setAuthorizationEnabled(true); + NamespaceService namespaceService = pulsar.getNamespaceService(); + CompletableFuture future = new CompletableFuture<>(); + future.complete(false); + doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); + + // Get the current semaphore first + Integer state1 = pulsar.getBrokerService().getLookupRequestSemaphore().availablePermits(); + AsyncResponse asyncResponse1 = mock(AsyncResponse.class); + // We used a nonexistent topic to test + destLookup.lookupTopicAsync(asyncResponse1, TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic2", false, null, null); + // Gets semaphore status + Integer state2 = pulsar.getBrokerService().getLookupRequestSemaphore().availablePermits(); + // If it is successfully released, it should be equal + assertEquals(state1, state2); + + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 2e584489c0675..c22e49e5fea80 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -45,6 +45,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -73,6 +74,7 @@ import org.apache.pulsar.common.naming.NamespaceBundles; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.LocalPolicies; @@ -95,6 +97,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "flaky") @@ -799,6 +802,30 @@ public void testModularLoadManagerRemoveBundleAndLoad() throws Exception { assertFalse(getResult.isPresent()); } + @DataProvider(name = "topicDomain") + public Object[] topicDomain() { + return new Object[]{ + TopicDomain.persistent.value(), + TopicDomain.non_persistent.value() + }; + } + + @Test(dataProvider = "topicDomain") + public void testCheckTopicExists(String topicDomain) throws Exception { + String topic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); + admin.topics().createNonPartitionedTopic(topic); + Awaitility.await().untilAsserted(() -> { + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get()); + }); + + String partitionedTopic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); + admin.topics().createPartitionedTopic(partitionedTopic, 5); + Awaitility.await().untilAsserted(() -> { + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get()); + }); + } + /** * 1. Manually trigger "LoadReportUpdaterTask" * 2. Registry another new zk-node-listener "waitForBrokerChangeNotice". diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java new file mode 100644 index 0000000000000..b446f9e902f2a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.qos; + +import static org.testng.Assert.assertEquals; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class AsyncTokenBucketTest { + private AtomicLong manualClockSource; + private MonotonicSnapshotClock clockSource; + + private AsyncTokenBucket asyncTokenBucket; + + @BeforeMethod + public void setup() { + manualClockSource = new AtomicLong(TimeUnit.SECONDS.toNanos(100)); + clockSource = requestSnapshot -> manualClockSource.get(); + } + + + private void incrementSeconds(int seconds) { + manualClockSource.addAndGet(TimeUnit.SECONDS.toNanos(seconds)); + } + + private void incrementMillis(long millis) { + manualClockSource.addAndGet(TimeUnit.MILLISECONDS.toNanos(millis)); + } + + @Test + void shouldAddTokensWithConfiguredRate() { + asyncTokenBucket = + AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + incrementSeconds(5); + assertEquals(asyncTokenBucket.getTokens(), 50); + incrementSeconds(1); + assertEquals(asyncTokenBucket.getTokens(), 60); + incrementSeconds(4); + assertEquals(asyncTokenBucket.getTokens(), 100); + + // No matter how long the period is, tokens do not go above capacity + incrementSeconds(5); + assertEquals(asyncTokenBucket.getTokens(), 100); + + // Consume all and verify none available and then wait 1 period and check replenished + asyncTokenBucket.consumeTokens(100); + assertEquals(asyncTokenBucket.tokens(true), 0); + incrementSeconds(1); + assertEquals(asyncTokenBucket.getTokens(), 10); + } + + @Test + void shouldCalculatePauseCorrectly() { + asyncTokenBucket = + AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource) + .build(); + incrementSeconds(5); + asyncTokenBucket.consumeTokens(100); + assertEquals(asyncTokenBucket.getTokens(), -50); + assertEquals(TimeUnit.NANOSECONDS.toMillis(asyncTokenBucket.calculateThrottlingDuration()), 5100); + } + + @Test + void shouldSupportFractionsWhenUpdatingTokens() { + asyncTokenBucket = + AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + incrementMillis(100); + assertEquals(asyncTokenBucket.getTokens(), 1); + } + + @Test + void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens() { + asyncTokenBucket = + AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + for (int i = 0; i < 150; i++) { + incrementMillis(1); + } + assertEquals(asyncTokenBucket.getTokens(), 1); + incrementMillis(150); + assertEquals(asyncTokenBucket.getTokens(), 3); + } + +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java index d4bc9eb925da7..3e88af9ec0be7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java @@ -21,7 +21,9 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; - +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; @@ -34,12 +36,9 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - public class ResourceGroupRateLimiterTest extends BrokerTestBase { + private static final long MESSAGE_SIZE_SERIALIZED = 41L; final String rgName = "testRG"; org.apache.pulsar.common.policies.data.ResourceGroup testAddRg = new org.apache.pulsar.common.policies.data.ResourceGroup(); @@ -53,7 +52,6 @@ protected void setup() throws Exception { conf.setMaxPendingPublishRequestsPerConnection(0); super.baseSetup(); prepareData(); - } @AfterClass(alwaysRun = true) @@ -123,7 +121,7 @@ private void testRateLimit() throws PulsarAdminException, PulsarClientException, try { // third one should succeed - messageId = producer.sendAsync(new byte[MESSAGE_SIZE]).get(100, TimeUnit.MILLISECONDS); + messageId = producer.sendAsync(new byte[MESSAGE_SIZE]).get(300, TimeUnit.MILLISECONDS); Assert.assertNotNull(messageId); } catch (TimeoutException e) { Assert.fail("should not fail"); @@ -133,6 +131,8 @@ private void testRateLimit() throws PulsarAdminException, PulsarClientException, admin.namespaces().removeNamespaceResourceGroup(namespaceName); deleteResourceGroup(rgName); + Thread.sleep(2000); + // No rate limits should be applied. for (int i = 0; i < 5; i++) { messageId = producer.sendAsync(new byte[MESSAGE_SIZE]).get(100, TimeUnit.MILLISECONDS); @@ -148,7 +148,7 @@ public void testResourceGroupPublishRateLimit() throws Exception { } private void prepareData() { - testAddRg.setPublishRateInBytes(Long.valueOf(MESSAGE_SIZE)); + testAddRg.setPublishRateInBytes(Long.valueOf(MESSAGE_SIZE_SERIALIZED)); testAddRg.setPublishRateInMsgs(1); testAddRg.setDispatchRateInMsgs(-1); testAddRg.setDispatchRateInBytes(Long.valueOf(-1)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java index 332cccc2d2c6a..0a1213ac1a860 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -117,7 +118,7 @@ public void testFilterEntriesForConsumerOfEntryFilter() throws Exception { int size = this.helper.filterEntriesForConsumer(entries, batchSizes, sendMessageInfo, null, cursor, false, null); assertEquals(size, 0); - verify(subscriptionDispatchRateLimiter).tryDispatchPermit(1, expectedBytePermits); + verify(subscriptionDispatchRateLimiter).consumeDispatchQuota(1, expectedBytePermits); } @Test @@ -279,7 +280,8 @@ public boolean canUnsubscribe(Consumer consumer) { } @Override - public CompletableFuture close() { + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { return null; } @@ -294,7 +296,8 @@ public CompletableFuture disconnectActiveConsumers(boolean isResetCursor) } @Override - public CompletableFuture disconnectAllConsumers(boolean isResetCursor) { + public CompletableFuture disconnectAllConsumers(boolean isResetCursor, + Optional assignedBrokerLookupData) { return null; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java index 7f5d3e0edf178..39be56e3f41cf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java @@ -19,12 +19,14 @@ package org.apache.pulsar.broker.service; import static org.mockito.Mockito.CALLS_REAL_METHODS; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; import static org.testng.Assert.assertEquals; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -43,8 +45,10 @@ public void beforeMethod() { subscription = mock(AbstractSubscription.class); when(brokerService.pulsar()).thenReturn(pulsarService); + doReturn(pulsarService).when(brokerService).getPulsar(); when(pulsarService.getConfiguration()).thenReturn(serviceConfiguration); when(brokerService.getBacklogQuotaManager()).thenReturn(backlogQuotaManager); + doReturn(AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK).when(pulsarService).getMonotonicSnapshotClock(); topic = mock(AbstractTopic.class, withSettings() .useConstructor("topic", brokerService) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java index a28b60bbae354..0a6cffc7685d4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java @@ -25,18 +25,27 @@ import java.util.List; +import java.util.Optional; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -526,4 +535,58 @@ public void testDynamicConfigurationTopicAutoCreationPartitionedWhenDefaultMoreT } } + @Test + public void testExtensibleLoadManagerImplInternalTopicAutoCreations() + throws PulsarAdminException, PulsarClientException { + pulsar.getConfiguration().setAllowAutoTopicCreation(true); + pulsar.getConfiguration().setAllowAutoTopicCreationType(TopicType.PARTITIONED); + pulsar.getConfiguration().setDefaultNumPartitions(3); + pulsar.getConfiguration().setMaxNumPartitionsPerPartitionedTopic(5); + final String namespaceName = NamespaceName.SYSTEM_NAMESPACE.toString(); + TenantInfoImpl tenantInfo = new TenantInfoImpl(); + tenantInfo.setAllowedClusters(Set.of(configClusterName)); + admin.tenants().createTenant("pulsar", tenantInfo); + admin.namespaces().createNamespace(namespaceName); + admin.topics().createNonPartitionedTopic(ServiceUnitStateChannelImpl.TOPIC); + admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC); + admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); + + // clear the topics to test the auto creation of non-persistent topics. + ConcurrentOpenHashMap>> topics = + pulsar.getBrokerService().getTopics(); + ConcurrentOpenHashMap>> oldTopics = new ConcurrentOpenHashMap<>(); + topics.forEach((key, val) -> oldTopics.put(key, val)); + topics.clear(); + + // The created persistent topic correctly can be found by + // pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + Producer producer = pulsarClient.newProducer().topic(ServiceUnitStateChannelImpl.TOPIC).create(); + + // The created non-persistent topics cannot be found, as we did topics.clear() + try { + pulsarClient.newProducer().topic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC).create(); + Assert.fail("Create should have failed."); + } catch (PulsarClientException.TopicDoesNotExistException e) { + // expected + } + try { + pulsarClient.newProducer().topic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC).create(); + Assert.fail("Create should have failed."); + } catch (PulsarClientException.TopicDoesNotExistException e) { + // expected + } + + oldTopics.forEach((key, val) -> topics.put(key, val)); + + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + List partitionedTopicList = admin.topics().getPartitionedTopicList(namespaceName); + assertEquals(partitionedTopicList.size(), 0); + }); + + producer.close(); + admin.namespaces().deleteNamespace(namespaceName); + admin.tenants().deleteTenant("pulsar"); + + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 35f68a7a9ca7f..f456a133d99b1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -61,7 +61,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; @@ -1482,81 +1481,6 @@ public void testMetricsProvider() throws IOException { Assert.assertTrue(sb.toString().contains("test_metrics")); } - @Test - public void testPublishRateLimiterMonitor() { - BrokerService.PublishRateLimiterMonitor monitor = new BrokerService.PublishRateLimiterMonitor("test"); - AtomicInteger checkCnt = new AtomicInteger(0); - AtomicInteger refreshCnt = new AtomicInteger(0); - monitor.startOrUpdate(100, checkCnt::incrementAndGet, refreshCnt::incrementAndGet); - Assert.assertEquals(monitor.getTickTimeMs(), 100); - Awaitility.await().until(() -> checkCnt.get() > 0); - Awaitility.await().until(() -> refreshCnt.get() > 0); - - monitor.startOrUpdate(500, checkCnt::incrementAndGet, refreshCnt::incrementAndGet); - Assert.assertEquals(monitor.getTickTimeMs(), 500); - checkCnt.set(0); - refreshCnt.set(0); - Awaitility.await().until(() -> checkCnt.get() > 0); - Awaitility.await().until(() -> refreshCnt.get() > 0); - - monitor.stop(); - Assert.assertEquals(monitor.getTickTimeMs(), 0); - } - - @Test - public void testDynamicBrokerPublisherThrottlingTickTimeMillis() throws Exception { - cleanup(); - conf.setBrokerPublisherThrottlingMaxMessageRate(1000); - conf.setBrokerPublisherThrottlingTickTimeMillis(100); - setup(); - - int prevTickMills = 100; - BrokerService.PublishRateLimiterMonitor monitor = pulsar.getBrokerService().brokerPublishRateLimiterMonitor; - Awaitility.await().until(() -> monitor.getTickTimeMs() == prevTickMills); - - int newTickMills = prevTickMills * 2; - admin.brokers().updateDynamicConfiguration("brokerPublisherThrottlingTickTimeMillis", - String.valueOf(newTickMills)); - Awaitility.await().until(() -> monitor.getTickTimeMs() == newTickMills); - - admin.brokers().updateDynamicConfiguration("brokerPublisherThrottlingTickTimeMillis", - String.valueOf(0)); - Awaitility.await().until(() -> monitor.getTickTimeMs() == 0); - - admin.brokers().updateDynamicConfiguration("brokerPublisherThrottlingTickTimeMillis", - String.valueOf(prevTickMills)); - Awaitility.await().until(() -> monitor.getTickTimeMs() == prevTickMills); - } - - @Test - public void testDynamicTopicPublisherThrottlingTickTimeMillis() throws Exception { - cleanup(); - conf.setPreciseTopicPublishRateLimiterEnable(false); - conf.setMaxPublishRatePerTopicInMessages(1000); - conf.setTopicPublisherThrottlingTickTimeMillis(100); - setup(); - - @Cleanup - Producer producer = pulsarClient.newProducer().topic("persistent://prop/ns-abc/test-topic").create(); - - int prevTickMills = 100; - BrokerService.PublishRateLimiterMonitor monitor = pulsar.getBrokerService().topicPublishRateLimiterMonitor; - Awaitility.await().until(() -> monitor.getTickTimeMs() == prevTickMills); - - int newTickMills = prevTickMills * 2; - admin.brokers().updateDynamicConfiguration("topicPublisherThrottlingTickTimeMillis", - String.valueOf(newTickMills)); - Awaitility.await().until(() -> monitor.getTickTimeMs() == newTickMills); - - admin.brokers().updateDynamicConfiguration("topicPublisherThrottlingTickTimeMillis", - String.valueOf(0)); - Awaitility.await().until(() -> monitor.getTickTimeMs() == 0); - - admin.brokers().updateDynamicConfiguration("topicPublisherThrottlingTickTimeMillis", - String.valueOf(prevTickMills)); - Awaitility.await().until(() -> monitor.getTickTimeMs() == prevTickMills); - } - @Test public void shouldNotPreventCreatingTopicWhenNonexistingTopicIsCached() throws Exception { // run multiple iterations to increase the chance of reproducing a race condition in the topic cache diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java index 2f128fe6270a5..33e797fcb219f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java @@ -19,9 +19,18 @@ package org.apache.pulsar.broker.service; import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.InjectedClientCnxClientBuilder; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.policies.data.SubscriptionStats; @@ -32,10 +41,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.net.InetSocketAddress; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - @Test(groups = "broker") public class EnableProxyProtocolTest extends BrokerTestBase { @@ -46,6 +51,15 @@ protected void setup() throws Exception { super.baseSetup(); } + protected PulsarClient newPulsarClient(String url, int intervalInSecs) throws PulsarClientException { + ClientBuilder clientBuilder = + PulsarClient.builder() + .serviceUrl(url) + .statsInterval(intervalInSecs, TimeUnit.SECONDS); + customizeNewPulsarClientBuilder(clientBuilder); + return createNewPulsarClient(clientBuilder); + } + @AfterClass(alwaysRun = true) @Override protected void cleanup() throws Exception { @@ -53,7 +67,7 @@ protected void cleanup() throws Exception { } @Test - public void testSimpleProduceAndConsume() throws PulsarClientException { + public void testSimpleProduceAndConsume() throws Exception { final String namespace = "prop/ns-abc"; final String topicName = "persistent://" + namespace + "/testSimpleProduceAndConsume"; final String subName = "my-subscriber-name"; @@ -76,30 +90,104 @@ public void testSimpleProduceAndConsume() throws PulsarClientException { } Assert.assertEquals(received, messages); + + // cleanup. + org.apache.pulsar.broker.service.Consumer serverConsumer = pulsar.getBrokerService().getTopicReference(topicName) + .get().getSubscription(subName).getConsumers().get(0); + ((ServerCnx) serverConsumer.cnx()).close(); + consumer.close(); + producer.close(); + admin.topics().delete(topicName); } @Test - public void testProxyProtocol() throws PulsarClientException, ExecutionException, InterruptedException, PulsarAdminException { + public void testProxyProtocol() throws Exception { final String namespace = "prop/ns-abc"; final String topicName = "persistent://" + namespace + "/testProxyProtocol"; final String subName = "my-subscriber-name"; - PulsarClientImpl client = (PulsarClientImpl) pulsarClient; - CompletableFuture cnx = client.getCnxPool().getConnection(InetSocketAddress.createUnresolved("localhost", pulsar.getBrokerListenPort().get())); - // Simulate the proxy protcol message - cnx.get().ctx().channel().writeAndFlush(Unpooled.copiedBuffer("PROXY TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes())); - pulsarClient.newConsumer().topic(topicName).subscriptionName(subName) - .subscribe(); - org.apache.pulsar.broker.service.Consumer c = pulsar.getBrokerService().getTopicReference(topicName).get().getSubscription(subName).getConsumers().get(0); - Awaitility.await().untilAsserted(() -> Assert.assertTrue(c.cnx().hasHAProxyMessage())); + + // Create a client that injected the protocol implementation. + ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); + PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, + (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + public void channelActive(ChannelHandlerContext ctx) throws Exception { + byte[] bs = "PROXY TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes(); + ctx.writeAndFlush(Unpooled.copiedBuffer(bs)); + super.channelActive(ctx); + } + }); + + // Verify the addr can be handled correctly. + testPubAndSub(topicName, subName, "198.51.100.22:35646", protocolClient); + + // cleanup. + admin.topics().delete(topicName); + } + + @Test(timeOut = 10000) + public void testPubSubWhenSlowNetwork() throws Exception { + final String namespace = "prop/ns-abc"; + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp"); + final String subName = "my-subscriber-name"; + + // Create a client that injected the protocol implementation. + ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); + PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, + (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + public void channelActive(ChannelHandlerContext ctx) throws Exception { + Thread task = new Thread(() -> { + try { + byte[] bs1 = "PROXY".getBytes(); + byte[] bs2 = " TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes(); + ctx.writeAndFlush(Unpooled.copiedBuffer(bs1)); + Thread.sleep(100); + ctx.writeAndFlush(Unpooled.copiedBuffer(bs2)); + super.channelActive(ctx); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + task.start(); + } + }); + + // Verify the addr can be handled correctly. + testPubAndSub(topicName, subName, "198.51.100.22:35646", protocolClient); + + // cleanup. + admin.topics().delete(topicName); + } + + private void testPubAndSub(String topicName, String subName, String expectedHostAndPort, + PulsarClientImpl pulsarClient) throws Exception { + // Verify: subscribe + org.apache.pulsar.client.api.Consumer clientConsumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(subName).subscribe(); + org.apache.pulsar.broker.service.Consumer serverConsumer = pulsar.getBrokerService() + .getTopicReference(topicName).get().getSubscription(subName).getConsumers().get(0); + Awaitility.await().untilAsserted(() -> Assert.assertTrue(serverConsumer.cnx().hasHAProxyMessage())); TopicStats topicStats = admin.topics().getStats(topicName); Assert.assertEquals(topicStats.getSubscriptions().size(), 1); SubscriptionStats subscriptionStats = topicStats.getSubscriptions().get(subName); Assert.assertEquals(subscriptionStats.getConsumers().size(), 1); - Assert.assertEquals(subscriptionStats.getConsumers().get(0).getAddress(), "198.51.100.22:35646"); + Assert.assertEquals(subscriptionStats.getConsumers().get(0).getAddress(), expectedHostAndPort); + + // Verify: producer register. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + TopicStats topicStats2 = admin.topics().getStats(topicName); + Assert.assertEquals(topicStats2.getPublishers().size(), 1); + Assert.assertEquals(topicStats2.getPublishers().get(0).getAddress(), expectedHostAndPort); + + // Verify: Pub & Sub + producer.send("1"); + Message msg = clientConsumer.receive(2, TimeUnit.SECONDS); + Assert.assertNotNull(msg); + Assert.assertEquals(msg.getValue(), "1"); + clientConsumer.acknowledge(msg); - pulsarClient.newProducer().topic(topicName).create(); - topicStats = admin.topics().getStats(topicName); - Assert.assertEquals(topicStats.getPublishers().size(), 1); - Assert.assertEquals(topicStats.getPublishers().get(0).getAddress(), "198.51.100.22:35646"); + // cleanup. + ((ServerCnx) serverConsumer.cnx()).close(); + producer.close(); + clientConsumer.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java index 173f772a7316f..27f72eac94254 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java @@ -139,12 +139,5 @@ public void testBlockByPublishRateLimiting() throws Exception { Awaitility.await().untilAsserted(() -> assertEquals(pulsar.getBrokerService().getPausedConnections(), 0)); - - // Resume message publish. - ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().enableCnxAutoRead(); - - flushFuture.get(); - Awaitility.await().untilAsserted(() -> - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0)); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 0dfa5cbb45454..ba680e4bcd74c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -158,8 +158,7 @@ public void setup() throws Exception { doReturn(new PulsarCommandSenderImpl(null, serverCnxWithOldVersion)) .when(serverCnxWithOldVersion).getCommandSender(); - NamespaceService nsSvc = mock(NamespaceService.class); - doReturn(nsSvc).when(pulsarTestContext.getPulsarService()).getNamespaceService(); + NamespaceService nsSvc = pulsarTestContext.getPulsarService().getNamespaceService(); doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class)); doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class)); doReturn(CompletableFuture.completedFuture(true)).when(nsSvc).checkTopicOwnership(any(TopicName.class)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java new file mode 100644 index 0000000000000..ab8d4dbe5cc01 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +@Test(groups = "broker") +@Slf4j +public class PersistentTopicInitializeDelayTest extends BrokerTestBase { + + @BeforeMethod + @Override + protected void setup() throws Exception { + conf.setTopicFactoryClassName(MyTopicFactory.class.getName()); + conf.setAllowAutoTopicCreation(true); + conf.setManagedLedgerMaxEntriesPerLedger(1); + conf.setBrokerDeleteInactiveTopicsEnabled(false); + conf.setTransactionCoordinatorEnabled(false); + conf.setTopicLoadTimeoutSeconds(30); + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicInitializeDelay() throws Exception { + admin.tenants().createTenant("public", TenantInfo.builder().allowedClusters(Set.of(configClusterName)).build()); + String namespace = "public/initialize-delay"; + admin.namespaces().createNamespace(namespace); + final String topicName = "persistent://" + namespace + "/testTopicInitializeDelay"; + admin.topics().createNonPartitionedTopic(topicName); + + admin.topicPolicies().setMaxConsumers(topicName, 10); + Awaitility.await().untilAsserted(() -> assertEquals(admin.topicPolicies().getMaxConsumers(topicName), 10)); + admin.topics().unload(topicName); + CompletableFuture> optionalFuture = pulsar.getBrokerService().getTopic(topicName, true); + + Optional topic = optionalFuture.get(15, TimeUnit.SECONDS); + assertTrue(topic.isPresent()); + } + + public static class MyTopicFactory implements TopicFactory { + @Override + public T create(String topic, ManagedLedger ledger, BrokerService brokerService, + Class topicClazz) { + try { + if (topicClazz == NonPersistentTopic.class) { + return (T) new NonPersistentTopic(topic, brokerService); + } else { + return (T) new MyPersistentTopic(topic, ledger, brokerService); + } + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + @Override + public void close() throws IOException { + // No-op + } + } + + public static class MyPersistentTopic extends PersistentTopic { + + private static AtomicInteger checkReplicationInvocationCount = new AtomicInteger(0); + + public MyPersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerService) { + super(topic, ledger, brokerService); + SystemTopicBasedTopicPoliciesService topicPoliciesService = + (SystemTopicBasedTopicPoliciesService) brokerService.getPulsar().getTopicPoliciesService(); + if (topicPoliciesService.getListeners().containsKey(TopicName.get(topic)) ) { + this.onUpdate(brokerService.getPulsar().getTopicPoliciesService().getTopicPoliciesIfExists(TopicName.get(topic))); + } + } + + protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { + try { + Thread.sleep(10 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + super.updateTopicPolicyByNamespacePolicy(namespacePolicies); + } + + public CompletableFuture checkReplication() { + if (TopicName.get(topic).getLocalName().equalsIgnoreCase("testTopicInitializeDelay")) { + checkReplicationInvocationCount.incrementAndGet(); + log.info("checkReplication, count = {}", checkReplicationInvocationCount.get()); + List configuredClusters = topicPolicies.getReplicationClusters().get(); + if (!(configuredClusters.size() == 1 && configuredClusters.contains(brokerService.pulsar().getConfiguration().getClusterName()))) { + try { + // this will cause the get topic timeout. + Thread.sleep(8 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("checkReplication error"); + } + } + return super.checkReplication(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index e0a13e103c647..04bf36eaa6645 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -211,10 +211,9 @@ public void setup() throws Exception { doReturn(ctx).when(serverCnx).ctx(); doReturn(CompletableFuture.completedFuture(true)).when(serverCnx).checkConnectionLiveness(); - NamespaceService nsSvc = mock(NamespaceService.class); + NamespaceService nsSvc = pulsarTestContext.getPulsarService().getNamespaceService(); NamespaceBundle bundle = mock(NamespaceBundle.class); doReturn(CompletableFuture.completedFuture(bundle)).when(nsSvc).getBundleAsync(any()); - doReturn(nsSvc).when(pulsarTestContext.getPulsarService()).getNamespaceService(); doReturn(true).when(nsSvc).isServiceUnitOwned(any()); doReturn(true).when(nsSvc).isServiceUnitActive(any()); doReturn(CompletableFuture.completedFuture(true)).when(nsSvc).isServiceUnitActiveAsync(any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisePublishLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisePublishLimiterTest.java deleted file mode 100644 index 9d5cfe5b5d28e..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisePublishLimiterTest.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.service; - -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; -import lombok.Cleanup; -import org.apache.pulsar.common.policies.data.PublishRate; -import org.testng.annotations.Test; - -public class PrecisePublishLimiterTest { - - @Test - void shouldResetMsgLimitAfterUpdate() { - @Cleanup - PrecisePublishLimiter precisePublishLimiter = new PrecisePublishLimiter(new PublishRate(), () -> { - }); - precisePublishLimiter.update(new PublishRate(1, 1)); - assertFalse(precisePublishLimiter.tryAcquire(99, 99)); - precisePublishLimiter.update(new PublishRate(-1, 100)); - assertTrue(precisePublishLimiter.tryAcquire(99, 99)); - } - - @Test - void shouldResetBytesLimitAfterUpdate() { - @Cleanup - PrecisePublishLimiter precisePublishLimiter = new PrecisePublishLimiter(new PublishRate(), () -> { - }); - precisePublishLimiter.update(new PublishRate(1, 1)); - assertFalse(precisePublishLimiter.tryAcquire(99, 99)); - precisePublishLimiter.update(new PublishRate(100, -1)); - assertTrue(precisePublishLimiter.tryAcquire(99, 99)); - } - - @Test - void shouldCloseResources() throws Exception { - for (int i = 0; i < 20000; i++) { - PrecisePublishLimiter precisePublishLimiter = new PrecisePublishLimiter(new PublishRate(100, 100), () -> { - }); - precisePublishLimiter.tryAcquire(99, 99); - precisePublishLimiter.close(); - } - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterDisableTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterDisableTest.java index 8dfed77814b35..ec952a7ca7734 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterDisableTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterDisableTest.java @@ -18,7 +18,10 @@ */ package org.apache.pulsar.broker.service; -import static org.testng.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.testng.annotations.Test; public class PublishRateLimiterDisableTest { @@ -26,7 +29,9 @@ public class PublishRateLimiterDisableTest { // GH issue #10603 @Test void shouldAlwaysAllowAcquire() { - PublishRateLimiterDisable publishRateLimiter = PublishRateLimiterDisable.DISABLED_RATE_LIMITER; - assertTrue(publishRateLimiter.tryAcquire(Integer.MAX_VALUE, Long.MAX_VALUE)); + PublishRateLimiter publishRateLimiter = new PublishRateLimiterImpl(AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK); + Producer producer = mock(Producer.class); + publishRateLimiter.handlePublishThrottling(producer, Integer.MAX_VALUE, Long.MAX_VALUE); + verify(producer, never()).incrementThrottleCount(); } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java index f3cb25e789f08..2c44ba7e23004 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java @@ -16,154 +16,105 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.pulsar.broker.service; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import io.netty.channel.EventLoopGroup; +import java.util.HashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PublishRate; -import org.apache.pulsar.common.util.RateLimiter; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.lang.reflect.Method; -import java.util.HashMap; -import java.util.concurrent.ScheduledFuture; - -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; - @Test(groups = "broker") public class PublishRateLimiterTest { private final String CLUSTER_NAME = "clusterName"; private final Policies policies = new Policies(); private final PublishRate publishRate = new PublishRate(10, 100); private final PublishRate newPublishRate = new PublishRate(20, 200); + private AtomicLong manualClockSource; - private PrecisePublishLimiter precisePublishLimiter; + private Producer producer; private PublishRateLimiterImpl publishRateLimiter; + private AtomicInteger throttleCount = new AtomicInteger(0); + @BeforeMethod public void setup() throws Exception { policies.publishMaxMessageRate = new HashMap<>(); policies.publishMaxMessageRate.put(CLUSTER_NAME, publishRate); - - precisePublishLimiter = new PrecisePublishLimiter(policies, CLUSTER_NAME, () -> System.out.print("Refresh permit")); - publishRateLimiter = new PublishRateLimiterImpl(policies, CLUSTER_NAME); + manualClockSource = new AtomicLong(TimeUnit.SECONDS.toNanos(100)); + publishRateLimiter = new PublishRateLimiterImpl(requestSnapshot -> manualClockSource.get()); + publishRateLimiter.update(policies, CLUSTER_NAME); + producer = mock(Producer.class); + throttleCount.set(0); + doAnswer(a -> { + throttleCount.incrementAndGet(); + return null; + }).when(producer).incrementThrottleCount(); + doAnswer(a -> { + throttleCount.decrementAndGet(); + return null; + }).when(producer).decrementThrottleCount(); + TransportCnx transportCnx = mock(TransportCnx.class); + when(producer.getCnx()).thenReturn(transportCnx); + BrokerService brokerService = mock(BrokerService.class); + when(transportCnx.getBrokerService()).thenReturn(brokerService); + EventLoopGroup eventLoopGroup = mock(EventLoopGroup.class); + when(brokerService.executor()).thenReturn(eventLoopGroup); + doReturn(null).when(eventLoopGroup).schedule(any(Runnable.class), anyLong(), any()); + incrementSeconds(1); } @AfterMethod public void cleanup() throws Exception { policies.publishMaxMessageRate.clear(); policies.publishMaxMessageRate = null; - precisePublishLimiter.close(); - publishRateLimiter.close(); + } + + private void incrementSeconds(int seconds) { + manualClockSource.addAndGet(TimeUnit.SECONDS.toNanos(seconds)); } @Test public void testPublishRateLimiterImplExceed() throws Exception { // increment not exceed - publishRateLimiter.incrementPublishCount(5, 50); - publishRateLimiter.checkPublishRate(); - assertFalse(publishRateLimiter.isPublishRateExceeded()); - publishRateLimiter.resetPublishCount(); + publishRateLimiter.handlePublishThrottling(producer, 5, 50); + assertEquals(throttleCount.get(), 0); + + incrementSeconds(1); // numOfMessages increment exceeded - publishRateLimiter.incrementPublishCount(11, 100); - publishRateLimiter.checkPublishRate(); - assertTrue(publishRateLimiter.isPublishRateExceeded()); - publishRateLimiter.resetPublishCount(); + publishRateLimiter.handlePublishThrottling(producer, 11, 100); + assertEquals(throttleCount.get(), 1); - // msgSizeInBytes increment exceeded - publishRateLimiter.incrementPublishCount(9, 110); - publishRateLimiter.checkPublishRate(); - assertTrue(publishRateLimiter.isPublishRateExceeded()); + incrementSeconds(1); + // msgSizeInBytes increment exceeded + publishRateLimiter.handlePublishThrottling(producer, 9, 110); + assertEquals(throttleCount.get(), 2); } @Test public void testPublishRateLimiterImplUpdate() { - publishRateLimiter.incrementPublishCount(11, 110); - publishRateLimiter.checkPublishRate(); - assertTrue(publishRateLimiter.isPublishRateExceeded()); + publishRateLimiter.handlePublishThrottling(producer, 11, 110); + assertEquals(throttleCount.get(), 1); // update + throttleCount.set(0); publishRateLimiter.update(newPublishRate); - publishRateLimiter.incrementPublishCount(11, 110); - publishRateLimiter.checkPublishRate(); - assertFalse(publishRateLimiter.isPublishRateExceeded()); - - } - - @Test - public void testPrecisePublishRateLimiterUpdate() { - assertFalse(precisePublishLimiter.tryAcquire(15, 150)); - - //update - precisePublishLimiter.update(newPublishRate); - assertTrue(precisePublishLimiter.tryAcquire(15, 150)); - } - - @Test - public void testPrecisePublishRateLimiterAcquire() throws Exception { - Class precisePublishLimiterClass = Class.forName("org.apache.pulsar.broker.service.PrecisePublishLimiter"); - Field topicPublishRateLimiterOnMessageField = precisePublishLimiterClass.getDeclaredField("topicPublishRateLimiterOnMessage"); - Field topicPublishRateLimiterOnByteField = precisePublishLimiterClass.getDeclaredField("topicPublishRateLimiterOnByte"); - topicPublishRateLimiterOnMessageField.setAccessible(true); - topicPublishRateLimiterOnByteField.setAccessible(true); - - RateLimiter topicPublishRateLimiterOnMessage = (RateLimiter)topicPublishRateLimiterOnMessageField.get( - precisePublishLimiter); - RateLimiter topicPublishRateLimiterOnByte = (RateLimiter)topicPublishRateLimiterOnByteField.get( - precisePublishLimiter); - - Method renewTopicPublishRateLimiterOnMessageMethod = topicPublishRateLimiterOnMessage.getClass().getDeclaredMethod("renew", null); - Method renewTopicPublishRateLimiterOnByteMethod = topicPublishRateLimiterOnByte.getClass().getDeclaredMethod("renew", null); - renewTopicPublishRateLimiterOnMessageMethod.setAccessible(true); - renewTopicPublishRateLimiterOnByteMethod.setAccessible(true); - - // running tryAcquire in order to lazyInit the renewTask - precisePublishLimiter.tryAcquire(1, 10); - - Field onMessageRenewTaskField = topicPublishRateLimiterOnMessage.getClass().getDeclaredField("renewTask"); - Field onByteRenewTaskField = topicPublishRateLimiterOnByte.getClass().getDeclaredField("renewTask"); - onMessageRenewTaskField.setAccessible(true); - onByteRenewTaskField.setAccessible(true); - ScheduledFuture onMessageRenewTask = (ScheduledFuture) onMessageRenewTaskField.get(topicPublishRateLimiterOnMessage); - ScheduledFuture onByteRenewTask = (ScheduledFuture) onByteRenewTaskField.get(topicPublishRateLimiterOnByte); - - onMessageRenewTask.cancel(false); - onByteRenewTask.cancel(false); - - // renewing the permits from previous tests - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - - // tryAcquire not exceeded - assertTrue(precisePublishLimiter.tryAcquire(1, 10)); - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - - // tryAcquire numOfMessages exceeded - assertFalse(precisePublishLimiter.tryAcquire(11, 100)); - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - - // tryAcquire msgSizeInBytes exceeded - assertFalse(precisePublishLimiter.tryAcquire(10, 101)); - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - - // tryAcquire exceeded exactly - assertFalse(precisePublishLimiter.tryAcquire(10, 100)); - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - renewTopicPublishRateLimiterOnMessageMethod.invoke(topicPublishRateLimiterOnMessage); - renewTopicPublishRateLimiterOnByteMethod.invoke(topicPublishRateLimiterOnByte); - - // tryAcquire not exceeded - assertTrue(precisePublishLimiter.tryAcquire(9, 99)); + publishRateLimiter.handlePublishThrottling(producer, 11, 110); + assertEquals(throttleCount.get(), 0); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java index 62ba0e83c8600..4296f3f416868 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java @@ -25,6 +25,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -90,14 +91,12 @@ public void testRemoveLocalClusterOnGlobalNamespace() throws Exception { admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r2", "r3")); - MockedPulsarServiceBaseTest - .retryStrategically((test) -> !pulsar1.getBrokerService().getTopics().containsKey(topicName), 50, 150); - - Assert.assertFalse(pulsar1.getBrokerService().getTopics().containsKey(topicName)); - Assert.assertFalse(producer1.isConnected()); - Assert.assertFalse(consumer1.isConnected()); - Assert.assertTrue(consumer2.isConnected()); - + Awaitility.await().atMost(1, TimeUnit.MINUTES).untilAsserted(() -> { + Assert.assertFalse(pulsar1.getBrokerService().getTopics().containsKey(topicName)); + Assert.assertFalse(producer1.isConnected()); + Assert.assertFalse(consumer1.isConnected()); + Assert.assertTrue(consumer2.isConnected()); + }); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java index b97210b009a3b..747ef3b7f5ce8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java @@ -33,6 +33,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,6 +60,7 @@ public void beforeMethod(Method m) throws Exception { @Override @BeforeClass(timeOut = 300000) public void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); super.setup(); } @@ -66,6 +68,7 @@ public void setup() throws Exception { @AfterClass(alwaysRun = true, timeOut = 300000) public void cleanup() throws Exception { super.cleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); } enum DispatchRateType { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 529fb923f5918..fe519827be74a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.ReplicatedSubscriptionsController; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -822,6 +823,66 @@ public void testWriteMarkerTaskOfReplicateSubscriptions(boolean isTopicPolicyEna pulsar1.getConfiguration().setForceDeleteNamespaceAllowed(false); } + @Test + public void testReplicatedSubscriptionWithCompaction() throws Exception { + final String namespace = BrokerTestUtil.newUniqueName("pulsar/replicatedsubscription"); + final String topicName = "persistent://" + namespace + "/testReplicatedSubscriptionWithCompaction"; + final String subName = "sub"; + + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + admin1.topics().createNonPartitionedTopic(topicName); + admin1.topicPolicies().setCompactionThreshold(topicName, 100 * 1024 * 1024L); + + @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(url1.toString()) + .statsInterval(0, TimeUnit.SECONDS).build(); + + Producer producer = client.newProducer(Schema.STRING).topic(topicName).create(); + producer.newMessage().key("K1").value("V1").send(); + producer.newMessage().key("K1").value("V2").send(); + producer.close(); + + createReplicatedSubscription(client, topicName, subName, true); + Awaitility.await().untilAsserted(() -> { + Map status = admin1.topics().getReplicatedSubscriptionStatus(topicName, subName); + assertTrue(status.get(topicName)); + }); + + Awaitility.await().untilAsserted(() -> { + PersistentTopic t1 = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(topicName, false).get().get(); + ReplicatedSubscriptionsController rsc1 = t1.getReplicatedSubscriptionController().get(); + Assert.assertTrue(rsc1.getLastCompletedSnapshotId().isPresent()); + assertEquals(t1.getPendingWriteOps().get(), 0L); + }); + + admin1.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin1.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("sub2") + .subscriptionType(SubscriptionType.Exclusive) + .readCompacted(true) + .subscribe(); + List result = new ArrayList<>(); + while (true) { + Message receive = consumer.receive(2, TimeUnit.SECONDS); + if (receive == null) { + break; + } + + result.add(receive.getValue()); + } + + Assert.assertEquals(result, List.of("V2")); + } + /** * Disable replication subscription. * Test scheduled task case. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 0f0440d24dde7..b6dd42d702860 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -2413,7 +2413,8 @@ public void testSubscribeBookieTimeout() throws Exception { "test" /* consumer name */, 0 /* avoid reseting cursor */); channel.writeInbound(subscribe1); - ByteBuf closeConsumer = Commands.newCloseConsumer(1 /* consumer id */, 2 /* request id */); + ByteBuf closeConsumer = Commands.newCloseConsumer(1 /* consumer id */, 2 /* request id */, + null /* assignedBrokerServiceUrl */, null /* assignedBrokerServiceUrlTls */); channel.writeInbound(closeConsumer); ByteBuf subscribe2 = Commands.newSubscribe(successTopicName, // diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java similarity index 72% rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java index c22ed41fc1533..721d049342552 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java @@ -18,63 +18,31 @@ */ package org.apache.pulsar.broker.service; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.awaitility.Awaitility; import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - @Test(groups = "broker") -public class PrecisTopicPublishRateThrottleTest extends BrokerTestBase{ +public class TopicPublishRateThrottleTest extends BrokerTestBase{ + @BeforeMethod(alwaysRun = true) @Override protected void setup() throws Exception { - //No-op + AsyncTokenBucket.switchToConsistentTokensView(); } + @AfterMethod(alwaysRun = true) @Override protected void cleanup() throws Exception { - //No-op - } - - @Test - public void testPrecisTopicPublishRateLimitingDisabled() throws Exception { - PublishRate publishRate = new PublishRate(1,10); - // disable precis topic publish rate limiting - conf.setPreciseTopicPublishRateLimiterEnable(false); - conf.setMaxPendingPublishRequestsPerConnection(0); - super.baseSetup(); - admin.namespaces().setPublishRate("prop/ns-abc", publishRate); - final String topic = "persistent://prop/ns-abc/testPrecisTopicPublishRateLimiting"; - org.apache.pulsar.client.api.Producer producer = pulsarClient.newProducer() - .topic(topic) - .producerName("producer-name") - .create(); - - Topic topicRef = pulsar.getBrokerService().getTopicReference(topic).get(); - Assert.assertNotNull(topicRef); - MessageId messageId = null; - try { - // first will be success - messageId = producer.sendAsync(new byte[10]).get(500, TimeUnit.MILLISECONDS); - Assert.assertNotNull(messageId); - // second will be success - messageId = producer.sendAsync(new byte[10]).get(500, TimeUnit.MILLISECONDS); - Assert.assertNotNull(messageId); - } catch (TimeoutException e) { - // No-op - } - Thread.sleep(1000); - try { - messageId = producer.sendAsync(new byte[10]).get(1, TimeUnit.SECONDS); - } catch (TimeoutException e) { - // No-op - } - Assert.assertNotNull(messageId); super.internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); } @Test @@ -103,7 +71,6 @@ public void testProducerBlockedByPrecisTopicPublishRateLimiting() throws Excepti } catch (TimeoutException e) { // No-op } - super.internalCleanup(); } @Test @@ -139,7 +106,6 @@ public void testPrecisTopicPublishRateLimitingProduceRefresh() throws Exception // No-op } Assert.assertNotNull(messageId); - super.internalCleanup(); } @Test @@ -164,9 +130,10 @@ public void testBrokerLevelPublishRateDynamicUpdate() throws Exception{ "" + rateInMsg)); Topic topicRef = pulsar.getBrokerService().getTopicReference(topic).get(); Assert.assertNotNull(topicRef); - PrecisePublishLimiter limiter = ((PrecisePublishLimiter) ((AbstractTopic) topicRef).topicPublishRateLimiter); - Awaitility.await().untilAsserted(() -> Assert.assertEquals(limiter.publishMaxMessageRate, rateInMsg)); - Assert.assertEquals(limiter.publishMaxByteRate, 0); + PublishRateLimiterImpl limiter = ((PublishRateLimiterImpl) ((AbstractTopic) topicRef).topicPublishRateLimiter); + Awaitility.await() + .untilAsserted(() -> Assert.assertEquals(limiter.getTokenBucketOnMessage().getRate(), rateInMsg)); + Assert.assertNull(limiter.getTokenBucketOnByte()); // maxPublishRatePerTopicInBytes admin.brokers().updateDynamicConfiguration("maxPublishRatePerTopicInBytes", "" + rateInByte); @@ -174,10 +141,10 @@ public void testBrokerLevelPublishRateDynamicUpdate() throws Exception{ .untilAsserted(() -> Assert.assertEquals(admin.brokers().getAllDynamicConfigurations().get("maxPublishRatePerTopicInBytes"), "" + rateInByte)); - Awaitility.await().untilAsserted(() -> Assert.assertEquals(limiter.publishMaxByteRate, rateInByte)); - Assert.assertEquals(limiter.publishMaxMessageRate, rateInMsg); + Awaitility.await() + .untilAsserted(() -> Assert.assertEquals(limiter.getTokenBucketOnByte().getRate(), rateInByte)); + Assert.assertEquals(limiter.getTokenBucketOnMessage().getRate(), rateInMsg); producer.close(); - super.internalCleanup(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java index a665681528114..402b5c4972ce2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java @@ -32,6 +32,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import io.netty.buffer.ByteBuf; import io.netty.channel.EventLoopGroup; import java.lang.reflect.Field; @@ -47,16 +48,24 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.BacklogQuotaManager; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactionServiceFactory; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @Slf4j @Test(groups = "broker") -public class MessageDuplicationTest { +public class MessageDuplicationTest extends BrokerTestBase { private static final int BROKER_DEDUPLICATION_ENTRIES_INTERVAL = 10; private static final int BROKER_DEDUPLICATION_MAX_NUMBER_PRODUCERS = 10; @@ -255,7 +264,9 @@ public void testIsDuplicateWithFailure() { BrokerService brokerService = mock(BrokerService.class); doReturn(eventLoopGroup).when(brokerService).executor(); doReturn(pulsarService).when(brokerService).pulsar(); + doReturn(pulsarService).when(brokerService).getPulsar(); doReturn(new BacklogQuotaManager(pulsarService)).when(brokerService).getBacklogQuotaManager(); + doReturn(AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK).when(pulsarService).getMonotonicSnapshotClock(); PersistentTopic persistentTopic = spyWithClassAndConstructorArgs(PersistentTopic.class, "topic-1", brokerService, managedLedger, messageDeduplication); @@ -440,4 +451,43 @@ public void completed(Exception e, long ledgerId, long entryId) { } }); } + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + this.conf.setBrokerDeduplicationEnabled(true); + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testMessageDeduplication() throws Exception { + String topicName = "persistent://prop/ns-abc/testMessageDeduplication"; + String producerName = "test-producer"; + Producer producer = pulsarClient + .newProducer(Schema.STRING) + .producerName(producerName) + .topic(topicName) + .create(); + final PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService() + .getTopicIfExists(topicName).get().orElse(null); + assertNotNull(persistentTopic); + final MessageDeduplication messageDeduplication = persistentTopic.getMessageDeduplication(); + assertFalse(messageDeduplication.getInactiveProducers().containsKey(producerName)); + producer.close(); + Awaitility.await().untilAsserted(() -> assertTrue(messageDeduplication.getInactiveProducers().containsKey(producerName))); + admin.topicPolicies().setDeduplicationStatus(topicName, false); + Awaitility.await().untilAsserted(() -> { + final Boolean deduplicationStatus = admin.topicPolicies().getDeduplicationStatus(topicName); + Assert.assertNotNull(deduplicationStatus); + Assert.assertFalse(deduplicationStatus); + }); + messageDeduplication.purgeInactiveProducers(); + assertTrue(messageDeduplication.getInactiveProducers().isEmpty()); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java index 1c4f88bc0273c..7b3daddcd9da0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java @@ -255,7 +255,7 @@ public void testFilter() throws Exception { int counter = 0; while (true) { - Message message = consumer.receive(1, TimeUnit.SECONDS); + Message message = consumer.receive(5, TimeUnit.SECONDS); if (message != null) { counter++; consumer.acknowledge(message); @@ -289,7 +289,7 @@ public void testFilter() throws Exception { counter = 0; while (true) { - Message message = consumer.receive(1, TimeUnit.SECONDS); + Message message = consumer.receive(5, TimeUnit.SECONDS); if (message != null) { counter++; consumer.acknowledge(message); @@ -321,7 +321,7 @@ public void testFilter() throws Exception { } counter = 0; while (true) { - Message message = consumer.receive(1, TimeUnit.SECONDS); + Message message = consumer.receive(5, TimeUnit.SECONDS); if (message != null) { counter++; consumer.acknowledge(message); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 80f9cd8913f64..abd00d374f32f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -120,25 +120,12 @@ protected void cleanup() throws Exception { @Test public void testPublishRateLimitedTimes() throws Exception { - checkPublishRateLimitedTimes(true); - checkPublishRateLimitedTimes(false); - } - - private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Exception { cleanup(); - if (preciseRateLimit) { - conf.setBrokerPublisherThrottlingTickTimeMillis(10000000); - conf.setMaxPublishRatePerTopicInMessages(1); - conf.setMaxPublishRatePerTopicInBytes(1); - conf.setBrokerPublisherThrottlingMaxMessageRate(100000); - conf.setBrokerPublisherThrottlingMaxByteRate(10000000); - } else { - conf.setBrokerPublisherThrottlingTickTimeMillis(1); - conf.setBrokerPublisherThrottlingMaxMessageRate(1); - conf.setBrokerPublisherThrottlingMaxByteRate(1); - } + conf.setMaxPublishRatePerTopicInMessages(1); + conf.setMaxPublishRatePerTopicInBytes(1); + conf.setBrokerPublisherThrottlingMaxMessageRate(100000); + conf.setBrokerPublisherThrottlingMaxByteRate(10000000); conf.setStatsUpdateFrequencyInSecs(100000000); - conf.setPreciseTopicPublishRateLimiterEnable(preciseRateLimit); setup(); String ns1 = "prop/ns-abc1" + UUID.randomUUID(); admin.namespaces().createNamespace(ns1, 1); @@ -180,15 +167,9 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep assertEquals(item.value, 1); return; } else if (item.tags.get("topic").equals(topicName3)) { - //When using precise rate limiting, we only trigger the rate limiting of the topic, - // so if the topic is not using the same connection, the rate limiting times will be 0 - //When using asynchronous rate limiting, we will trigger the broker-level rate limiting, - // and all connections will be limited at this time. - if (preciseRateLimit) { - assertEquals(item.value, 0); - } else { - assertEquals(item.value, 1); - } + // We only trigger the rate limiting of the topic, so if the topic is not using + // the same connection, the rate limiting times will be 0 + assertEquals(item.value, 0); return; } fail("should not fail"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 6959d8dd04861..eba7f1e8c73c3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -106,6 +106,7 @@ import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStoreProvider; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -1378,7 +1379,7 @@ public void testGetConnectExceptionForAckMsgWhenCnxIsNull() throws Exception { producer.newMessage().value(Bytes.toBytes(i)).send(); } ClientCnx cnx = (ClientCnx) MethodUtils.invokeMethod(consumer, true, "cnx"); - MethodUtils.invokeMethod(consumer, true, "connectionClosed", cnx); + MethodUtils.invokeMethod(consumer, true, "connectionClosed", cnx, Optional.empty(), Optional.empty()); Message message = consumer.receive(); Transaction transaction = pulsarClient @@ -1849,4 +1850,62 @@ public void testReadCommittedWithReadCompacted() throws Exception{ Assert.assertEquals(messages, List.of("V2", "V3")); } + + @Test + public void testReadCommittedWithCompaction() throws Exception{ + final String namespace = "tnx/ns-prechecks"; + final String topic = "persistent://" + namespace + "/test_transaction_topic" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + admin.topics().createNonPartitionedTopic(topic); + + admin.topicPolicies().setCompactionThreshold(topic, 100 * 1024 * 1024); + + @Cleanup + Producer producer = this.pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + producer.newMessage().key("K1").value("V1").send(); + + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn).key("K2").value("V2").send(); + producer.newMessage(txn).key("K3").value("V3").send(); + txn.commit().get(); + + producer.newMessage().key("K1").value("V4").send(); + + Transaction txn2 = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn2).key("K2").value("V5").send(); + producer.newMessage(txn2).key("K3").value("V6").send(); + txn2.commit().get(); + + admin.topics().triggerCompaction(topic); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topic).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Consumer consumer = this.pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Exclusive) + .readCompacted(true) + .subscribe(); + List result = new ArrayList<>(); + while (true) { + Message receive = consumer.receive(2, TimeUnit.SECONDS); + if (receive == null) { + break; + } + + result.add(receive.getValue()); + } + + Assert.assertEquals(result, List.of("V4", "V5", "V6")); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index a50363c861caa..e5ad910cb1f10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -18,9 +18,18 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.when; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import static org.testng.AssertJUnit.fail; +import java.util.List; +import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.BrokerService; @@ -30,8 +39,13 @@ import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; +import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.TopicMessageIdImpl; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; @@ -179,4 +193,121 @@ public void testCloseTransactionBufferWhenTimeout() throws Exception { Assert.assertTrue(f.isCompletedExceptionally()); } + /** + * This test mainly test the following two point: + * 1. `getLastMessageIds` will get max read position. + * Send two message |1:0|1:1|; mock max read position as |1:0|; `getLastMessageIds` will get |1:0|. + * 2. `getLastMessageIds` will wait Transaction buffer recover completely. + * Mock `checkIfTBRecoverCompletely` return an exception, `getLastMessageIds` will fail too. + * Mock `checkIfTBRecoverCompletely` return null, `getLastMessageIds` will get correct result. + */ + @Test + public void testGetMaxPositionAfterTBReady() throws Exception { + // 1. Prepare test environment. + String topic = "persistent://" + NAMESPACE1 + "/testGetMaxReadyPositionAfterTBReady"; + // 1.1 Mock component. + TransactionBuffer transactionBuffer = Mockito.spy(TransactionBuffer.class); + when(transactionBuffer.checkIfTBRecoverCompletely(anyBoolean())) + // Handle producer will check transaction buffer recover completely. + .thenReturn(CompletableFuture.completedFuture(null)) + // If the Transaction buffer failed to recover, we can not get the correct last max read id. + .thenReturn(CompletableFuture.failedFuture(new Throwable("Mock fail"))) + // If the transaction buffer recover successfully, the max read position can be acquired successfully. + .thenReturn(CompletableFuture.completedFuture(null)); + TransactionBufferProvider transactionBufferProvider = Mockito.spy(TransactionBufferProvider.class); + Mockito.doReturn(transactionBuffer).when(transactionBufferProvider).newTransactionBuffer(any()); + TransactionBufferProvider originalTBProvider = getPulsarServiceList().get(0).getTransactionBufferProvider(); + Mockito.doReturn(transactionBufferProvider).when(getPulsarServiceList().get(0)).getTransactionBufferProvider(); + // 2. Building producer and consumer. + admin.topics().createNonPartitionedTopic(topic); + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName("sub") + .subscribe(); + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .create(); + // 3. Send message and test the exception can be handled as expected. + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().send(); + producer.newMessage().send(); + Mockito.doReturn(new PositionImpl(messageId.getLedgerId(), messageId.getEntryId())) + .when(transactionBuffer).getMaxReadPosition(); + try { + consumer.getLastMessageIds(); + fail(); + } catch (PulsarClientException exception) { + assertTrue(exception.getMessage().contains("Failed to recover Transaction Buffer.")); + } + List messageIdList = consumer.getLastMessageIds(); + assertEquals(messageIdList.size(), 1); + TopicMessageIdImpl actualMessageID = (TopicMessageIdImpl) messageIdList.get(0); + assertEquals(messageId.getLedgerId(), actualMessageID.getLedgerId()); + assertEquals(messageId.getEntryId(), actualMessageID.getEntryId()); + // 4. Clean resource + Mockito.doReturn(originalTBProvider).when(getPulsarServiceList().get(0)).getTransactionBufferProvider(); + } + + /** + * Add a E2E test for the get last message ID. It tests 4 cases. + *

+ * 1. Only normal messages in the topic. + * 2. There are ongoing transactions, last message ID will not be updated until transaction end. + * 3. Aborted transaction will make the last message ID be updated as expected. + * 4. Committed transaction will make the last message ID be updated as expected. + *

+ */ + @Test + public void testGetLastMessageIdsWithOngoingTransactions() throws Exception { + // 1. Prepare environment + String topic = "persistent://" + NAMESPACE1 + "/testGetLastMessageIdsWithOngoingTransactions"; + String subName = "my-subscription"; + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .create(); + Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName(subName) + .subscribe(); + + // 2. Test last max read position can be required correctly. + // 2.1 Case1: send 3 original messages. |1:0|1:1|1:2| + MessageIdImpl expectedLastMessageID = null; + for (int i = 0; i < 3; i++) { + expectedLastMessageID = (MessageIdImpl) producer.newMessage().send(); + } + assertMessageId(consumer, expectedLastMessageID, 0); + // 2.2 Case2: send 2 ongoing transactional messages and 2 original messages. + // |1:0|1:1|1:2|txn1->1:3|1:4|txn2->1:5|1:6|. + Transaction txn1 = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build() + .get(); + Transaction txn2 = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build() + .get(); + producer.newMessage(txn1).send(); + MessageIdImpl expectedLastMessageID1 = (MessageIdImpl) producer.newMessage().send(); + producer.newMessage(txn2).send(); + MessageIdImpl expectedLastMessageID2 = (MessageIdImpl) producer.newMessage().send(); + // 2.2.1 Last message ID will not change when txn1 and txn2 do not end. + assertMessageId(consumer, expectedLastMessageID, 0); + // 2.2.2 Last message ID will update to 1:4 when txn1 committed. + txn1.commit().get(5, TimeUnit.SECONDS); + assertMessageId(consumer, expectedLastMessageID1, 0); + // 2.2.3 Last message ID will update to 1:6 when txn2 aborted. + txn2.abort().get(5, TimeUnit.SECONDS); + // Todo: We can not ignore the marker's position in this fix. + assertMessageId(consumer, expectedLastMessageID2, 2); + } + + private void assertMessageId(Consumer consumer, MessageIdImpl expected, int entryOffset) throws Exception { + TopicMessageIdImpl actual = (TopicMessageIdImpl) consumer.getLastMessageIds().get(0); + assertEquals(expected.getEntryId(), actual.getEntryId() - entryOffset); + assertEquals(expected.getLedgerId(), actual.getLedgerId()); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 1684b2ca138e8..864b481b72a8a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -36,6 +36,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; @@ -270,9 +271,10 @@ public void testTransactionBufferClientTimeout() throws Exception { CompletableFuture completableFuture = new CompletableFuture<>(); ClientCnx clientCnx = mock(ClientCnx.class); completableFuture.complete(clientCnx); - when(((PulsarClientImpl)mockClient).getConnection(anyString())).thenReturn(completableFuture); - when(((PulsarClientImpl)mockClient).getConnection(anyString(), anyInt())).thenReturn(completableFuture); - when(((PulsarClientImpl)mockClient).getConnection(any(), any(), anyInt())).thenReturn(completableFuture); + when(mockClient.getConnection(anyString())).thenReturn(completableFuture); + when(mockClient.getConnection(anyString(), anyInt())).thenReturn( + CompletableFuture.completedFuture(Pair.of(clientCnx, false))); + when(mockClient.getConnection(any(), any(), anyInt())).thenReturn(completableFuture); ChannelHandlerContext cnx = mock(ChannelHandlerContext.class); when(clientCnx.ctx()).thenReturn(cnx); Channel channel = mock(Channel.class); @@ -324,10 +326,9 @@ public void testTransactionBufferChannelUnActive() throws PulsarServerException PulsarClientImpl mockClient = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); when(mockClient.getCnxPool()).thenReturn(connectionPool); - CompletableFuture completableFuture = new CompletableFuture<>(); ClientCnx clientCnx = mock(ClientCnx.class); - completableFuture.complete(clientCnx); - when(((PulsarClientImpl)mockClient).getConnection(anyString(), anyInt())).thenReturn(completableFuture); + when(mockClient.getConnection(anyString(), anyInt())).thenReturn( + CompletableFuture.completedFuture(Pair.of(clientCnx, false))); ChannelHandlerContext cnx = mock(ChannelHandlerContext.class); when(clientCnx.ctx()).thenReturn(cnx); Channel channel = mock(Channel.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java index 278cdbac1f09d..633671420e5fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; @@ -57,9 +58,9 @@ public void testRequestCredits() throws PulsarServerException { when(namespaceService.getBundleAsync(any())).thenReturn(CompletableFuture.completedFuture(mock(NamespaceBundle.class))); Optional opData = Optional.empty(); when(namespaceService.getOwnerAsync(any())).thenReturn(CompletableFuture.completedFuture(opData)); - when(((PulsarClientImpl)pulsarClient).getConnection(anyString(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(mock(ClientCnx.class))); - when(((PulsarClientImpl)pulsarClient).getConnection(anyString())) + when(pulsarClient.getConnection(anyString(), anyInt())) + .thenReturn(CompletableFuture.completedFuture(Pair.of(mock(ClientCnx.class), false))); + when(pulsarClient.getConnection(anyString())) .thenReturn(CompletableFuture.completedFuture(mock(ClientCnx.class))); TransactionBufferHandlerImpl handler = spy(new TransactionBufferHandlerImpl(pulsarService, null, 1000, 3000)); doNothing().when(handler).endTxn(any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java index 0bafc7c57c957..f9aa17ea3c451 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.zookeeper.KeeperException.Code; import org.awaitility.Awaitility; @@ -498,4 +499,48 @@ public void testCleanupEmptyTopicAuthenticationMap() throws Exception { .get().auth_policies.getTopicAuthentication().containsKey(topic)); }); } + + @Test + public void testCleanupEmptySubscriptionAuthenticationMap() throws Exception { + Map authParams = new HashMap<>(); + authParams.put("tlsCertFile", getTlsFileForClient("admin.cert")); + authParams.put("tlsKeyFile", getTlsFileForClient("admin.key-pk8")); + Authentication authTls = new AuthenticationTls(); + authTls.configure(authParams); + internalSetup(authTls); + + admin.clusters().createCluster("test", ClusterData.builder().build()); + admin.tenants().createTenant("p1", + new TenantInfoImpl(Collections.emptySet(), new HashSet<>(admin.clusters().getClusters()))); + String namespace = "p1/ns1"; + admin.namespaces().createNamespace("p1/ns1"); + + // grant permission1 and permission2 + String subscription = "test-sub-1"; + String role1 = "test-user-1"; + String role2 = "test-user-2"; + Set roles = new HashSet<>(); + roles.add(role1); + roles.add(role2); + admin.namespaces().grantPermissionOnSubscription(namespace, subscription, roles); + Optional policies = pulsar.getPulsarResources().getNamespaceResources().getPolicies(NamespaceName.get(namespace)); + assertTrue(policies.isPresent()); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().containsKey(subscription)); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role1)); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role2)); + + // revoke permission1 + admin.namespaces().revokePermissionOnSubscription(namespace, subscription, role1); + policies = pulsar.getPulsarResources().getNamespaceResources().getPolicies(NamespaceName.get(namespace)); + assertTrue(policies.isPresent()); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().containsKey(subscription)); + assertFalse(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role1)); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role2)); + + // revoke permission2 + admin.namespaces().revokePermissionOnSubscription(namespace, subscription, role2); + policies = pulsar.getPulsarResources().getNamespaceResources().getPolicies(NamespaceName.get(namespace)); + assertTrue(policies.isPresent()); + assertFalse(policies.get().auth_policies.getSubscriptionAuthentication().containsKey(subscription)); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientErrorsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientErrorsTest.java index 61c7a98602b69..705b171929be6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientErrorsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientErrorsTest.java @@ -270,7 +270,7 @@ private void consumerCreatedThenFailsRetryTimeout(String topic) throws Exception if (subscribeCount == 1) { ctx.writeAndFlush(Commands.newSuccess(subscribe.getRequestId())); // Trigger reconnect - ctx.writeAndFlush(Commands.newCloseConsumer(subscribe.getConsumerId(), -1)); + ctx.writeAndFlush(Commands.newCloseConsumer(subscribe.getConsumerId(), -1, null, null)); } else if (subscribeCount != 2) { // Respond to subsequent requests to prevent timeouts ctx.writeAndFlush(Commands.newSuccess(subscribe.getRequestId())); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java new file mode 100644 index 0000000000000..d29dd4f7061b8 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import io.netty.channel.EventLoopGroup; +import java.util.concurrent.ThreadFactory; +import org.apache.pulsar.client.impl.ClientBuilderImpl; +import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.client.impl.ConnectionPool; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.netty.EventLoopUtil; + +public class InjectedClientCnxClientBuilder { + + public static PulsarClientImpl create(final ClientBuilderImpl clientBuilder, + final ClientCnxFactory clientCnxFactory) throws Exception { + ClientConfigurationData conf = clientBuilder.getClientConfigurationData(); + ThreadFactory threadFactory = new ExecutorProvider + .ExtendedThreadFactory("pulsar-client-io", Thread.currentThread().isDaemon()); + EventLoopGroup eventLoopGroup = + EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), conf.isEnableBusyWait(), threadFactory); + + // Inject into ClientCnx. + ConnectionPool pool = new ConnectionPool(conf, eventLoopGroup, + () -> clientCnxFactory.generate(conf, eventLoopGroup)); + + return new PulsarClientImpl(conf, eventLoopGroup, pool); + } + + public interface ClientCnxFactory { + + ClientCnx generate(ClientConfigurationData conf, EventLoopGroup eventLoopGroup); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index 4f4affc39d316..360d27f64133d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +67,7 @@ public class MessageDispatchThrottlingTest extends ProducerConsumerBase { @BeforeClass @Override protected void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); this.conf.setClusterName("test"); super.internalSetup(); super.producerBaseSetup(); @@ -75,6 +77,7 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { super.internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); } @AfterMethod(alwaysRun = true) @@ -246,20 +249,12 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr // create producer and topic Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 5; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0 - || topic.getDispatchRateLimiter().get().getDispatchRateOnByte() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0 + || topic.getDispatchRateLimiter().get().getDispatchRateOnByte() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); int numMessages = 500; @@ -387,19 +382,11 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio // create producer and topic Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 5; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); final int numProducedMessages = 20; @@ -498,7 +485,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT * * @throws Exception */ - @Test(timeOut = 5000) + @Test(timeOut = 10000) public void testRateLimitingMultipleConsumers() throws Exception { log.info("-- Starting {} test --", methodName); @@ -516,19 +503,11 @@ public void testRateLimitingMultipleConsumers() throws Exception { // create producer and topic Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 5; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); final int numProducedMessages = 500; @@ -536,11 +515,17 @@ public void testRateLimitingMultipleConsumers() throws Exception { final AtomicInteger totalReceived = new AtomicInteger(0); ConsumerBuilder consumerBuilder = pulsarClient.newConsumer().topic(topicName) + .receiverQueueSize(1) .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Shared).messageListener((c1, msg) -> { Assert.assertNotNull(msg, "Message cannot be null"); String receivedMessage = new String(msg.getData()); log.debug("Received message [{}] in the listener", receivedMessage); totalReceived.incrementAndGet(); + try { + c1.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } }); Consumer consumer1 = consumerBuilder.subscribe(); Consumer consumer2 = consumerBuilder.subscribe(); @@ -558,10 +543,10 @@ public void testRateLimitingMultipleConsumers() throws Exception { } // it can make sure that consumer had enough time to consume message but couldn't consume due to throttling - Thread.sleep(500); + Thread.sleep(1000); - // consumer should not have received all published message due to message-rate throttling - Assert.assertNotEquals(totalReceived.get(), numProducedMessages); + // rate limiter should have limited messages with at least 10% accuracy (or 2 messages if messageRate is low) + Assert.assertEquals(totalReceived.get(), messageRate, Math.max(messageRate / 10, 2)); consumer1.close(); consumer2.close(); @@ -593,19 +578,11 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { Producer producer = pulsarClient.newProducer().topic(topicName).enableBatching(true) .batchingMaxPublishDelay(1, TimeUnit.SECONDS).batchingMaxMessages(messagesPerBatch).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 5; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); final AtomicInteger totalReceived = new AtomicInteger(0); @@ -732,20 +709,12 @@ public void testMessageByteRateThrottlingCombined(SubscriptionType subscription) // create producer and topic Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 5; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0 - && topic.getDispatchRateLimiter().get().getDispatchRateOnByte() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0 + || topic.getDispatchRateLimiter().get().getDispatchRateOnByte() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); final int numProducedMessages = 200; @@ -813,20 +782,12 @@ public void testGlobalNamespaceThrottling() throws Exception { // create producer and topic Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 5; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0 - || topic.getDispatchRateLimiter().get().getDispatchRateOnByte() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0 + || topic.getDispatchRateLimiter().get().getDispatchRateOnByte() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); int numMessages = 500; @@ -1148,19 +1109,11 @@ public void testRelativeMessageRateLimitingThrottling(SubscriptionType subscript // create producer and topic Producer producer = pulsarClient.newProducer().topic(topicName).enableBatching(false).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - boolean isMessageRateUpdate = false; - int retry = 10; - for (int i = 0; i < retry; i++) { - if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0) { - isMessageRateUpdate = true; - break; - } else { - if (i != retry - 1) { - Thread.sleep(100); - } - } - } - Assert.assertTrue(isMessageRateUpdate); + + Awaitility.await() + .ignoreExceptions() + .until(() -> topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); Thread.sleep(2000); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java index 8365b7a555703..956b834e33435 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java @@ -140,21 +140,21 @@ private void doFindBrokerWithListenerName(boolean useHttp) throws Exception { LookupService lookupService = useHttp ? new HttpLookupService(conf, eventExecutors) : new BinaryProtoLookupService((PulsarClientImpl) this.pulsarClient, lookupUrl.toString(), "internal", false, this.executorService); + TopicName topicName = TopicName.get("persistent://public/default/test"); + // test request 1 { - CompletableFuture> future = - lookupService.getBroker(TopicName.get("persistent://public/default/test")); - Pair result = future.get(10, TimeUnit.SECONDS); - Assert.assertEquals(result.getKey(), brokerAddress); - Assert.assertEquals(result.getValue(), brokerAddress); + var result = lookupService.getBroker(topicName).get(10, TimeUnit.SECONDS); + Assert.assertEquals(result.getLogicalAddress(), brokerAddress); + Assert.assertEquals(result.getPhysicalAddress(), brokerAddress); + Assert.assertEquals(result.isUseProxy(), false); } // test request 2 { - CompletableFuture> future = - lookupService.getBroker(TopicName.get("persistent://public/default/test")); - Pair result = future.get(10, TimeUnit.SECONDS); - Assert.assertEquals(result.getKey(), brokerAddress); - Assert.assertEquals(result.getValue(), brokerAddress); + var result = lookupService.getBroker(topicName).get(10, TimeUnit.SECONDS); + Assert.assertEquals(result.getLogicalAddress(), brokerAddress); + Assert.assertEquals(result.getPhysicalAddress(), brokerAddress); + Assert.assertEquals(result.isUseProxy(), false); } } @@ -187,12 +187,11 @@ public void testHttpLookupRedirect() throws Exception { doReturn(CompletableFuture.completedFuture(optional), CompletableFuture.completedFuture(optional2)) .when(namespaceService).getBrokerServiceUrlAsync(any(), any()); - CompletableFuture> future = - lookupService.getBroker(TopicName.get("persistent://public/default/test")); - - Pair result = future.get(10, TimeUnit.SECONDS); - Assert.assertEquals(result.getKey(), address); - Assert.assertEquals(result.getValue(), address); + var result = + lookupService.getBroker(TopicName.get("persistent://public/default/test")).get(10, TimeUnit.SECONDS); + Assert.assertEquals(result.getLogicalAddress(), address); + Assert.assertEquals(result.getPhysicalAddress(), address); + Assert.assertEquals(result.isUseProxy(), false); } @AfterMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 9036d82d84f01..02de11a2bcc95 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -20,7 +20,7 @@ import static org.awaitility.Awaitility.await; import com.google.common.collect.Sets; -import java.time.Duration; +import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.broker.BrokerTestUtil; @@ -47,7 +47,7 @@ public class SubscriptionMessageDispatchThrottlingTest extends MessageDispatchTh * @param subscription * @throws Exception */ - @Test(dataProvider = "subscriptionAndDispatchRateType", timeOut = 5000) + @Test(dataProvider = "subscriptionAndDispatchRateType", timeOut = 30000) public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscription, DispatchRateType dispatchRateType) throws Exception { log.info("-- Starting {} test --", methodName); @@ -143,7 +143,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr * @param subscription * @throws Exception */ - @Test(dataProvider = "subscriptions", timeOut = 5000) + @Test(dataProvider = "subscriptions", timeOut = 30000) public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); @@ -217,7 +217,7 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio log.info("-- Exiting {} test --", methodName); } - @Test(dataProvider = "subscriptions", timeOut = 30000, invocationCount = 15) + @Test(dataProvider = "subscriptions", timeOut = 30000) private void testMessageNotDuplicated(SubscriptionType subscription) throws Exception { int brokerRate = 1000; int topicRate = 5000; @@ -272,7 +272,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - Awaitility.await().atMost(Duration.ofMillis(500)).untilAsserted(() -> { + Awaitility.await().untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); Assert.assertTrue(brokerDispatchRateLimiter != null && brokerDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -319,7 +319,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce * @param subscription * @throws Exception */ - @Test(dataProvider = "subscriptions", timeOut = 5000) + @Test(dataProvider = "subscriptions", timeOut = 30000) public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); @@ -450,7 +450,7 @@ private void testDispatchRate(SubscriptionType subscription, Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - Awaitility.await().atMost(Duration.ofMillis(500)).untilAsserted(() -> { + Awaitility.await().untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); Assert.assertTrue(brokerDispatchRateLimiter != null && brokerDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -525,7 +525,7 @@ public void testMultiLevelDispatch(SubscriptionType subscription) throws Excepti * @param subscription * @throws Exception */ - @Test(dataProvider = "subscriptions", timeOut = 8000) + @Test(dataProvider = "subscriptions", timeOut = 30000) public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); @@ -538,6 +538,11 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); final int byteRate = 1000; admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + byteRate); + + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(pulsar.getConfiguration().getDispatchThrottlingRateInByte(), byteRate); + }); + admin.namespaces().createNamespace(namespace1, Sets.newHashSet("test")); admin.namespaces().createNamespace(namespace2, Sets.newHashSet("test")); @@ -571,7 +576,7 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri Producer producer1 = pulsarClient.newProducer().topic(topicName1).create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2).create(); - Awaitility.await().atMost(Duration.ofMillis(500)).untilAsserted(() -> { + Awaitility.await().untilAsserted(() -> { DispatchRateLimiter rateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); Assert.assertTrue(rateLimiter != null && rateLimiter.getDispatchRateOnByte() > 0); @@ -604,7 +609,7 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri * * @throws Exception */ - @Test(timeOut = 5000) + @Test(timeOut = 30000) public void testRateLimitingMultipleConsumers() throws Exception { log.info("-- Starting {} test --", methodName); @@ -690,7 +695,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { } - @Test(dataProvider = "subscriptions", timeOut = 5000) + @Test(dataProvider = "subscriptions", timeOut = 30000) public void testClusterRateLimitingConfiguration(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); @@ -867,7 +872,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { log.info("-- Exiting {} test --", methodName); } - @Test(dataProvider = "subscriptions", timeOut = 11000) + @Test(dataProvider = "subscriptions", timeOut = 30000) public void testClosingRateLimiter(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); @@ -908,7 +913,7 @@ public void testClosingRateLimiter(SubscriptionType subscription) throws Excepti producer.close(); consumer.close(); - sub.disconnect().get(); + sub.close(true, Optional.empty()).get(); // Make sure that the rate limiter is closed Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), -1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java index 28eef0326cce3..c2715de986ad8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java @@ -104,6 +104,7 @@ import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -173,26 +174,15 @@ public void testDisconnectClientWithoutClosingConnection() throws Exception { doAnswer(invocationOnMock -> cons1.getState()).when(consumer1).getState(); doAnswer(invocationOnMock -> cons1.getClientCnx()).when(consumer1).getClientCnx(); doAnswer(invocationOnMock -> cons1.cnx()).when(consumer1).cnx(); - doAnswer(invocationOnMock -> { - cons1.connectionClosed((ClientCnx) invocationOnMock.getArguments()[0]); - return null; - }).when(consumer1).connectionClosed(any()); + doAnswer(InvocationOnMock::callRealMethod).when(consumer1).connectionClosed(any(), any(), any()); ProducerImpl producer1 = spy(prod1); doAnswer(invocationOnMock -> prod1.getState()).when(producer1).getState(); doAnswer(invocationOnMock -> prod1.getClientCnx()).when(producer1).getClientCnx(); doAnswer(invocationOnMock -> prod1.cnx()).when(producer1).cnx(); - doAnswer(invocationOnMock -> { - prod1.connectionClosed((ClientCnx) invocationOnMock.getArguments()[0]); - return null; - }).when(producer1).connectionClosed(any()); ProducerImpl producer2 = spy(prod2); doAnswer(invocationOnMock -> prod2.getState()).when(producer2).getState(); doAnswer(invocationOnMock -> prod2.getClientCnx()).when(producer2).getClientCnx(); doAnswer(invocationOnMock -> prod2.cnx()).when(producer2).cnx(); - doAnswer(invocationOnMock -> { - prod2.connectionClosed((ClientCnx) invocationOnMock.getArguments()[0]); - return null; - }).when(producer2).connectionClosed(any()); ClientCnx clientCnx = producer1.getClientCnx(); @@ -223,11 +213,11 @@ public void testDisconnectClientWithoutClosingConnection() throws Exception { // let server send signal to close-connection and client close the connection Thread.sleep(1000); // [1] Verify: producer1 must get connectionClosed signal - verify(producer1, atLeastOnce()).connectionClosed(any()); + verify(producer1, atLeastOnce()).connectionClosed(any(), any(), any()); // [2] Verify: consumer1 must get connectionClosed signal - verify(consumer1, atLeastOnce()).connectionClosed(any()); + verify(consumer1, atLeastOnce()).connectionClosed(any(), any(), any()); // [3] Verify: producer2 should have not received connectionClosed signal - verify(producer2, never()).connectionClosed(any()); + verify(producer2, never()).connectionClosed(any(), any(), any()); // sleep for sometime to let other disconnected producer and consumer connect again: but they should not get // connected with same broker as that broker is already out from active-broker list @@ -247,7 +237,7 @@ public void testDisconnectClientWithoutClosingConnection() throws Exception { pulsar.getNamespaceService().unloadNamespaceBundle((NamespaceBundle) bundle2).join(); // let producer2 give some time to get disconnect signal and get disconnected Thread.sleep(200); - verify(producer2, atLeastOnce()).connectionClosed(any()); + verify(producer2, atLeastOnce()).connectionClosed(any(), any(), any()); // producer1 must not be able to connect again assertNull(prod1.getClientCnx()); @@ -1083,4 +1073,41 @@ public void testManagedLedgerLazyCursorLedgerCreation() throws Exception { }); } + @Test + public void testSharedConsumerUnsubscribe() throws Exception { + String topic = "persistent://my-property/my-ns/sharedUnsubscribe"; + String sub = "my-subscriber-name"; + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer().topic(topic).subscriptionType(SubscriptionType.Shared) + .subscriptionName(sub).subscribe(); + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer().topic(topic).subscriptionType(SubscriptionType.Shared) + .subscriptionName(sub).subscribe(); + try { + consumer1.unsubscribe(); + fail("should have failed as consumer-2 is already connected"); + } catch (Exception e) { + // Ok + } + + consumer1.unsubscribe(true); + try { + consumer2.unsubscribe(true); + } catch (PulsarClientException.NotConnectedException e) { + // Ok. consumer-2 is already disconnected with force unsubscription + } + assertFalse(consumer1.isConnected()); + assertFalse(consumer2.isConnected()); + } + + @Test(dataProvider = "subType") + public void testUnsubscribeForce(SubscriptionType type) throws Exception { + String topic = "persistent://my-property/my-ns/sharedUnsubscribe"; + String sub = "my-subscriber-name"; + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer().topic(topic).subscriptionType(type) + .subscriptionName(sub).subscribe(); + consumer1.unsubscribe(true); + assertFalse(consumer1.isConnected()); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java index ad1955f08d779..1c0ae5547d53b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java @@ -18,9 +18,6 @@ */ package org.apache.pulsar.client.impl; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertNotSame; import static org.testng.Assert.assertTrue; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -34,14 +31,12 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.pulsar.broker.service.Producer; -import org.apache.pulsar.broker.service.PublishRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.common.policies.data.PublishRate; -import org.awaitility.Awaitility; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -53,6 +48,7 @@ public class MessagePublishThrottlingTest extends ProducerConsumerBase { @BeforeMethod @Override protected void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); this.conf.setClusterName("test"); this.conf.setTopicPublisherThrottlingTickTimeMillis(1); this.conf.setBrokerPublisherThrottlingTickTimeMillis(1); @@ -64,6 +60,7 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { super.internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); } /** @@ -86,16 +83,9 @@ public void testSimplePublishMessageThrottling() throws Exception { ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer().topic(topicName) .maxPendingMessages(30000).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); - // (1) verify message-rate is -1 initially - Assert.assertEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // enable throttling admin.namespaces().setPublishRate(namespace, publishMsgRate); - retryStrategically((test) -> - !topic.getTopicPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - Assert.assertNotEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); Producer prod = topic.getProducers().values().iterator().next(); // reset counter @@ -112,11 +102,6 @@ public void testSimplePublishMessageThrottling() throws Exception { // disable throttling publishMsgRate.publishThrottlingRateInMsg = -1; admin.namespaces().setPublishRate(namespace, publishMsgRate); - retryStrategically((test) -> - topic.getTopicPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - Assert.assertEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // reset counter prod.updateRates(); @@ -150,16 +135,9 @@ public void testSimplePublishByteThrottling() throws Exception { // create producer and topic ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); - // (1) verify message-rate is -1 initially - Assert.assertEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // enable throttling admin.namespaces().setPublishRate(namespace, publishMsgRate); - retryStrategically((test) -> - !topic.getTopicPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - Assert.assertNotEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); Producer prod = topic.getProducers().values().iterator().next(); // reset counter @@ -176,9 +154,6 @@ public void testSimplePublishByteThrottling() throws Exception { // disable throttling publishMsgRate.publishThrottlingRateInByte = -1; admin.namespaces().setPublishRate(namespace, publishMsgRate); - retryStrategically((test) -> topic.getTopicPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), 5, - 200); - Assert.assertEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // reset counter prod.updateRates(); @@ -214,8 +189,6 @@ public void testBrokerPublishMessageThrottling() throws Exception { .enableBatching(false) .maxPendingMessages(30000).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); - // (1) verify message-rate is -1 initially - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // enable throttling admin.brokers(). @@ -223,19 +196,11 @@ public void testBrokerPublishMessageThrottling() throws Exception { "brokerPublisherThrottlingMaxMessageRate", Integer.toString(messageRate)); - retryStrategically( - (test) -> - (topic.getBrokerPublishRateLimiter() != PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - log.info("Get broker configuration: brokerTick {}, MaxMessageRate {}, MaxByteRate {}", pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxMessageRate(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxByteRate()); - Assert.assertNotEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - Producer prod = topic.getProducers().values().iterator().next(); // reset counter prod.updateRates(); @@ -252,11 +217,6 @@ public void testBrokerPublishMessageThrottling() throws Exception { // disable throttling admin.brokers() .updateDynamicConfiguration("brokerPublisherThrottlingMaxMessageRate", Integer.toString(0)); - retryStrategically((test) -> - topic.getBrokerPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // reset counter prod.updateRates(); @@ -293,26 +253,16 @@ public void testBrokerPublishByteThrottling() throws Exception { .enableBatching(false) .maxPendingMessages(30000).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); - // (1) verify byte-rate is -1 disabled - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // enable throttling admin.brokers() .updateDynamicConfiguration("brokerPublisherThrottlingMaxByteRate", Long.toString(byteRate)); - retryStrategically( - (test) -> - (topic.getBrokerPublishRateLimiter() != PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - log.info("Get broker configuration after enable: brokerTick {}, MaxMessageRate {}, MaxByteRate {}", pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxMessageRate(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxByteRate()); - Assert.assertNotEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - Producer prod = topic.getProducers().values().iterator().next(); // reset counter prod.updateRates(); @@ -331,18 +281,12 @@ public void testBrokerPublishByteThrottling() throws Exception { // disable throttling admin.brokers() .updateDynamicConfiguration("brokerPublisherThrottlingMaxByteRate", Long.toString(0)); - retryStrategically((test) -> - topic.getBrokerPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); log.info("Get broker configuration after disable: brokerTick {}, MaxMessageRate {}, MaxByteRate {}", pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxMessageRate(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxByteRate()); - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - // reset counter prod.updateRates(); for (int i = 0; i < numMessage; i++) { @@ -385,21 +329,12 @@ public void testBrokerTopicPublishByteThrottling() throws Exception { .enableBatching(false) .maxPendingMessages(30000).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); - // (1) verify both broker and topic limiter is disabled - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - Assert.assertEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); // enable broker and topic throttling admin.namespaces().setPublishRate(namespace, topicPublishMsgRate); - Awaitility.await().untilAsserted(() -> { - assertNotEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - }); admin.brokers().updateDynamicConfiguration("brokerPublisherThrottlingMaxByteRate", Long.toString(brokerByteRate)); - Awaitility.await().untilAsserted(() -> { - assertNotSame(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - }); log.info("Get broker configuration after enable: brokerTick {}, MaxMessageRate {}, MaxByteRate {}", pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(), @@ -440,15 +375,7 @@ public void testBrokerTopicPublishByteThrottling() throws Exception { producers.add(iProducer); topics.add(iTopic); - // verify both broker and topic limiter is enabled - Assert.assertNotEquals(iTopic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - admin.namespaces().setPublishRate(namespace, topicPublishMsgRate); - retryStrategically((test) -> - !iTopic.getTopicPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); - Assert.assertNotEquals(iTopic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); } List> topicRatesCounter = Lists.newArrayListWithExpectedSize(3); @@ -486,10 +413,6 @@ public void testBrokerTopicPublishByteThrottling() throws Exception { topicPublishMsgRate.publishThrottlingRateInByte = -1; admin.namespaces().setPublishRate(namespace, topicPublishMsgRate); - Awaitility.await().untilAsserted(() -> - assertEquals(topic.getTopicPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER) - ); - // reset counter prod.updateRates(); for (int i = 0; i < numMessage; i++) { @@ -505,18 +428,12 @@ public void testBrokerTopicPublishByteThrottling() throws Exception { // disable broker throttling, expected no throttling. admin.brokers() .updateDynamicConfiguration("brokerPublisherThrottlingMaxByteRate", Long.toString(0)); - retryStrategically((test) -> - topic.getBrokerPublishRateLimiter().equals(PublishRateLimiter.DISABLED_RATE_LIMITER), - 5, - 200); log.info("Get broker configuration after disable: brokerTick {}, MaxMessageRate {}, MaxByteRate {}", pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxMessageRate(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxByteRate()); - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - // reset counter prod.updateRates(); for (int i = 0; i < numMessage; i++) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java index d588ac8626f7b..ab273913fde29 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java @@ -125,7 +125,7 @@ public CompletableFuture getConnection(String topic) { result.completeExceptionally(new IOException("New connections are rejected.")); return result; } else { - return super.getConnection(topic, getCnxPool().genRandomKeyToSelectCon()); + return super.getConnection(topic); } } @@ -192,7 +192,7 @@ public void disconnectProducerAndRejectReconnecting(ProducerImpl producer) th // make the existing connection between the producer and broker to break by explicitly closing it ClientCnx cnx = producer.cnx(); - producer.connectionClosed(cnx); + producer.connectionClosed(cnx, Optional.empty(), Optional.empty()); cnx.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicPublishThrottlingInitTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicPublishThrottlingInitTest.java index 1b8284c31e67e..fe2e6ec9670dc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicPublishThrottlingInitTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicPublishThrottlingInitTest.java @@ -21,13 +21,10 @@ import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import org.apache.pulsar.broker.service.Producer; -import org.apache.pulsar.broker.service.PublishRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.ProducerConsumerBase; -import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -74,16 +71,12 @@ public void testBrokerPublishMessageThrottlingInit() throws Exception { .enableBatching(false) .maxPendingMessages(30000).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); - // (1) verify message-rate is initialized when value configured in broker - Assert.assertNotEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); log.info("Get broker configuration: brokerTick {}, MaxMessageRate {}, MaxByteRate {}", pulsar.getConfiguration().getBrokerPublisherThrottlingTickTimeMillis(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxMessageRate(), pulsar.getConfiguration().getBrokerPublisherThrottlingMaxByteRate()); - Assert.assertNotEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER); - Producer prod = topic.getProducers().values().iterator().next(); // reset counter prod.updateRates(); @@ -100,8 +93,6 @@ public void testBrokerPublishMessageThrottlingInit() throws Exception { // disable throttling admin.brokers() .updateDynamicConfiguration("brokerPublisherThrottlingMaxMessageRate", Integer.toString(0)); - Awaitility.await().untilAsserted(() -> - Assert.assertEquals(topic.getBrokerPublishRateLimiter(), PublishRateLimiter.DISABLED_RATE_LIMITER)); // reset counter prod.updateRates(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index d5a1eca51e40d..9eda547968388 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -26,7 +26,6 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; - import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.ByteBuf; @@ -648,9 +647,9 @@ public static Object[][] retainNullKey() { @Test(dataProvider = "retainNullKey") public void testKeyLessMessagesPassThrough(boolean retainNullKey) throws Exception { - conf.setTopicCompactionRemainNullKey(retainNullKey); + conf.setTopicCompactionRetainNullKey(retainNullKey); restartBroker(); - FieldUtils.writeDeclaredField(compactor, "topicCompactionRemainNullKey", retainNullKey, true); + FieldUtils.writeDeclaredField(compactor, "topicCompactionRetainNullKey", retainNullKey, true); String topic = "persistent://my-property/use/my-ns/my-topic1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java index 1e8b26beee38a..9882b15450e40 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java @@ -20,6 +20,8 @@ import static org.apache.pulsar.common.util.PortManager.nextLockedFreePort; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.assertNotNull; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; @@ -33,6 +35,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -41,6 +44,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.WorkerInfo; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -242,6 +246,18 @@ public void testFunctionsCreation() throws Exception { log.info(" -------- Start test function : {}", functionName); + int finalI = i; + Awaitility.await().atMost(1, TimeUnit.MINUTES).pollInterval(1, TimeUnit.SECONDS).untilAsserted(() -> { + final PulsarWorkerService workerService = ((PulsarWorkerService) fnWorkerServices[finalI]); + final LeaderService leaderService = workerService.getLeaderService(); + assertNotNull(leaderService); + if (leaderService.isLeader()) { + assertTrue(true); + } else { + final WorkerInfo workerInfo = workerService.getMembershipManager().getLeader(); + assertTrue(workerInfo != null && !workerInfo.getWorkerId().equals(workerService.getWorkerConfig().getWorkerId())); + } + }); pulsarAdmins[i].functions().createFunctionWithUrl( functionConfig, jarFilePathUrl ); diff --git a/pulsar-broker/src/test/resources/prometheus_metrics_sample.txt b/pulsar-broker/src/test/resources/prometheus_metrics_sample.txt index 2022fbd800000..7cf8d3e7167d7 100644 --- a/pulsar-broker/src/test/resources/prometheus_metrics_sample.txt +++ b/pulsar-broker/src/test/resources/prometheus_metrics_sample.txt @@ -284,8 +284,6 @@ caffeine_cache_load_duration_seconds_count{cluster="use",cache="bookies-racks-da caffeine_cache_load_duration_seconds_sum{cluster="use",cache="bookies-racks-data"} 0.0 caffeine_cache_load_duration_seconds_count{cluster="use",cache="global-zk-exists"} 0.0 caffeine_cache_load_duration_seconds_sum{cluster="use",cache="global-zk-exists"} 0.0 -# TYPE pulsar_broker_throttled_connections_global_limit gauge -pulsar_broker_throttled_connections_global_limit{cluster="use"} 0.0 # TYPE process_cpu_seconds_total counter process_cpu_seconds_total{cluster="use"} 101.68 # TYPE process_start_time_seconds gauge diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java index c67ad08c83631..d24d674c018e9 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java @@ -73,6 +73,31 @@ public interface Consumer extends Closeable, MessageAcknowledger { */ CompletableFuture unsubscribeAsync(); + + /** + * Unsubscribe the consumer. + * + *

This call blocks until the consumer is unsubscribed. + * + *

Unsubscribing will the subscription to be deleted and all the + * data retained can potentially be deleted as well. + * + *

The operation will fail when performed on a shared subscription + * where multiple consumers are currently connected. + * + * @param force forcefully unsubscribe by disconnecting connected consumers. + * @throws PulsarClientException if the operation fails + */ + void unsubscribe(boolean force) throws PulsarClientException; + + /** + * Asynchronously unsubscribe the consumer. + * + * @see Consumer#unsubscribe() + * @param force forcefully unsubscribe by disconnecting connected consumers. + * @return {@link CompletableFuture} to track the operation + */ + CompletableFuture unsubscribeAsync(boolean force); /** * Receives a single message. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java index 9409eefe2e0f0..007308ec7ab46 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java @@ -658,6 +658,10 @@ public NotConnectedException() { public NotConnectedException(long sequenceId) { super("Not connected to broker", sequenceId); } + + public NotConnectedException(String msg) { + super(msg); + } } /** diff --git a/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java b/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java index 5778b0701a460..cbb704de138e4 100644 --- a/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java +++ b/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java @@ -564,7 +564,7 @@ private boolean getKeyAndDecryptData(MessageMetadata msgMetadata, ByteBuffer pay if (storedSecretKey != null) { // Taking a small performance hit here if the hash collides. When it - // retruns a different key, decryption fails. At this point, we would + // returns a different key, decryption fails. At this point, we would // call decryptDataKey to refresh the cache and come here again to decrypt. if (decryptData(storedSecretKey, msgMetadata, payload, targetBuffer)) { // If decryption succeeded, we can already return diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java index 646f4ef0f50cf..0ea56e4430951 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java @@ -485,6 +485,7 @@ public CmdClusters(Supplier admin) { jcommander.addCommand("delete", new Delete()); jcommander.addCommand("list", new List()); jcommander.addCommand("update-peer-clusters", new UpdatePeerClusters()); + jcommander.addCommand("get-cluster-migration", new GetClusterMigration()); jcommander.addCommand("update-cluster-migration", new UpdateClusterMigration()); jcommander.addCommand("get-peer-clusters", new GetPeerClusters()); jcommander.addCommand("get-failure-domain", new GetFailureDomain()); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 1aa905c647876..508642e63ae2b 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -32,6 +32,9 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.buffer.Unpooled; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -93,10 +96,12 @@ @Parameters(commandDescription = "Operations on persistent topics") public class CmdTopics extends CmdBase { private final CmdTopics.PartitionedLookup partitionedLookup; + private final CmdTopics.DeleteCmd deleteCmd; public CmdTopics(Supplier admin) { super("topics", admin); partitionedLookup = new PartitionedLookup(); + deleteCmd = new DeleteCmd(); jcommander.addCommand("list", new ListCmd()); jcommander.addCommand("list-partitioned-topics", new PartitionedTopicListCmd()); jcommander.addCommand("permissions", new Permissions()); @@ -105,7 +110,7 @@ public CmdTopics(Supplier admin) { jcommander.addCommand("lookup", new Lookup()); jcommander.addCommand("partitioned-lookup", partitionedLookup); jcommander.addCommand("bundle-range", new GetBundleRange()); - jcommander.addCommand("delete", new DeleteCmd()); + jcommander.addCommand("delete", deleteCmd); jcommander.addCommand("truncate", new TruncateCmd()); jcommander.addCommand("unload", new UnloadCmd()); jcommander.addCommand("subscriptions", new ListSubscriptions()); @@ -714,9 +719,12 @@ void run() throws Exception { + "And the application is not able to connect to the topic(delete then re-create with same name) again " + "if the schema auto uploading is disabled. Besides, users should to use the truncate cmd to clean up " + "data of the topic instead of delete cmd if users continue to use this topic later.") - private class DeleteCmd extends CliCommand { - @Parameter(description = "persistent://tenant/namespace/topic", required = true) - private java.util.List params; + protected class DeleteCmd extends CliCommand { + @Parameter(description = "Provide either a single topic in the format 'persistent://tenant/namespace/topic', " + + "or a path to a file containing a list of topics, e.g., 'path://resources/topics.txt'. " + + "This parameter is required.", + required = true) + java.util.List params; @Parameter(names = { "-f", "--force" }, description = "Close all producer/consumer/replicator and delete topic forcefully") @@ -726,10 +734,50 @@ private class DeleteCmd extends CliCommand { + "but the parameter is invalid and the schema is always deleted", hidden = true) private boolean deleteSchema = false; + @Parameter(names = {"-r", "regex"}, + description = "Use a regex expression to match multiple topics for deletion.") + boolean regex = false; + + @Parameter(names = {"--from-file"}, description = "Read a list of topics from a file for deletion.") + boolean readFromFile; + + @Override - void run() throws PulsarAdminException { - String topic = validateTopicName(params); - getTopics().delete(topic, force); + void run() throws PulsarAdminException, IOException { + if (readFromFile && regex) { + throw new ParameterException("Could not apply regex when read topics from file."); + } + if (readFromFile) { + String path = checkArgument(params); + List topicsFromFile = Files.readAllLines(Path.of(path)); + for (String t : topicsFromFile) { + try { + getTopics().delete(t, force); + } catch (Exception e) { + print("Failed to delete topic: " + t + ". Exception: " + e); + } + } + } else { + String topic = validateTopicName(params); + if (regex) { + String namespace = TopicName.get(topic).getNamespace(); + List topics = getTopics().getList(namespace); + topics = topics.stream().filter(s -> s.matches(topic)).toList(); + for (String t : topics) { + try { + getTopics().delete(t, force); + } catch (Exception e) { + print("Failed to delete topic: " + t + ". Exception: " + e); + } + } + } else { + try { + getTopics().delete(topic, force); + } catch (Exception e) { + print("Failed to delete topic: " + topic + ". Exception: " + e); + } + } + } } } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdClusters.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdClusters.java index a09dce8cd8516..f94ae7bb9f747 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdClusters.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdClusters.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; @@ -122,4 +123,10 @@ public void testListCmd() throws Exception { System.setOut(defaultSystemOut); } } + + @Test + public void testGetClusterMigration() throws Exception { + cmdClusters.run(new String[]{"get-cluster-migration", "test_cluster"}); + verify(clusters, times(1)).getClusterMigration("test_cluster"); + } } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index a1c3d6f902cad..ced1c989b118b 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -20,12 +20,15 @@ import static org.apache.pulsar.common.naming.TopicName.DEFAULT_NAMESPACE; import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; @@ -33,12 +36,15 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.client.admin.Lookup; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Schemas; import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -58,19 +64,23 @@ public class TestCmdTopics { private PulsarAdmin pulsarAdmin; private CmdTopics cmdTopics; private Lookup mockLookup; + private Topics mockTopics; private CmdTopics.PartitionedLookup partitionedLookup; + private CmdTopics.DeleteCmd deleteCmd; @BeforeMethod public void setup() throws Exception { pulsarAdmin = Mockito.mock(PulsarAdmin.class); - Topics mockTopics = mock(Topics.class); + mockTopics = mock(Topics.class); when(pulsarAdmin.topics()).thenReturn(mockTopics); Schemas mockSchemas = mock(Schemas.class); when(pulsarAdmin.schemas()).thenReturn(mockSchemas); mockLookup = mock(Lookup.class); when(pulsarAdmin.lookups()).thenReturn(mockLookup); + when(pulsarAdmin.topics()).thenReturn(mockTopics); cmdTopics = spy(new CmdTopics(() -> pulsarAdmin)); partitionedLookup = spy(cmdTopics.getPartitionedLookup()); + deleteCmd = spy(cmdTopics.getDeleteCmd()); } @AfterMethod(alwaysRun = true) @@ -160,4 +170,94 @@ public void testPartitionedLookupSortByBroker() throws Exception { partitionedLookup.sortByBroker = true; verify(mockLookup).lookupPartitionedTopic(eq(topic.toString())); } + @Test + public void testRunDeleteSingleTopic() throws PulsarAdminException, IOException { + // Setup: Specify a single topic to delete + deleteCmd.params = List.of("persistent://tenant/namespace/topic"); + + // Act: Run the delete command + deleteCmd.run(); + + // Assert: Verify that the delete method was called once for the specified topic + verify(mockTopics, times(1)).delete("persistent://tenant/namespace/topic", false); + } + + @Test + public void testRunDeleteMultipleTopics() throws PulsarAdminException, IOException { + // Setup: Specify a regex to delete multiple topics + deleteCmd.params = List.of("persistent://tenant/namespace/.*"); + deleteCmd.regex = true; + + // Mock: Simulate the return of multiple topics that match the regex + when(mockTopics.getList("tenant/namespace")).thenReturn(List.of( + "persistent://tenant/namespace/topic1", + "persistent://tenant/namespace/topic2")); + + // Act: Run the delete command + deleteCmd.run(); + + // Assert: Verify that the delete method was called once for each of the matching topics + verify(mockTopics, times(1)).getList("tenant/namespace"); + verify(mockTopics, times(1)).delete("persistent://tenant/namespace/topic1", false); + verify(mockTopics, times(1)).delete("persistent://tenant/namespace/topic2", false); + } + + @Test + public void testRunDeleteTopicsFromFile() throws PulsarAdminException, IOException { + // Setup: Create a temporary file and write some topics to it + Path tempFile = Files.createTempFile("topics", ".txt"); + List topics = List.of( + "persistent://tenant/namespace/topic1", + "persistent://tenant/namespace/topic2"); + Files.write(tempFile, topics); + + // Setup: Specify the temporary file as input for the delete command + deleteCmd.params = List.of(tempFile.toString()); + deleteCmd.readFromFile = true; + + // Act: Run the delete command + deleteCmd.run(); + + // Assert: Verify that the delete method was called once for each topic in the file + for (String topic : topics) { + verify(mockTopics, times(1)).delete(topic, false); + } + + // Cleanup: Delete the temporary file + Files.delete(tempFile); + } + + @Test + public void testRunDeleteTopicsFromFileWithException() throws PulsarAdminException, IOException { + // Setup: Create a temporary file and write some topics to it. + // Configure the delete method of mockTopics to throw a PulsarAdminException on any input. + doThrow(new PulsarAdminException("mock fail")).when(mockTopics).delete(anyString(), anyBoolean()); + Path tempFile = Files.createTempFile("topics", ".txt"); + List topics = List.of( + "persistent://tenant/namespace/topic1", + "persistent://tenant/namespace/topic2"); + Files.write(tempFile, topics); + + // Setup: Specify the temporary file as input for the delete command + deleteCmd.params = List.of(tempFile.toString()); + deleteCmd.readFromFile = true; + + // Act: Run the delete command + // Since we have configured the delete method of mockTopics to throw an exception when called, + // an exception should be thrown here. + deleteCmd.run(); + + // Assert: Verify that the delete method was called once for each topic in the file, + // even if one of them threw an exception. + // This proves that the program continues to attempt to delete the other topics + // even if an exception occurred while deleting a topic. + for (String topic : topics) { + verify(mockTopics, times(1)).delete(topic, false); + } + + // Cleanup: Delete the temporary file and recreate the mockTopics. + Files.delete(tempFile); + mockTopics = mock(Topics.class); + } + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 8ceb8e44975c8..bdf00844c1cd2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.mutable.MutableObject; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SchemaSerializationException; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; @@ -58,7 +57,7 @@ public class BinaryProtoLookupService implements LookupService { private final String listenerName; private final int maxLookupRedirects; - private final ConcurrentHashMap>> + private final ConcurrentHashMap> lookupInProgress = new ConcurrentHashMap<>(); private final ConcurrentHashMap> @@ -99,11 +98,11 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException { * topic-name * @return broker-socket-address that serves given topic */ - public CompletableFuture> getBroker(TopicName topicName) { + public CompletableFuture getBroker(TopicName topicName) { final MutableObject newFutureCreated = new MutableObject<>(); try { return lookupInProgress.computeIfAbsent(topicName, tpName -> { - CompletableFuture> newFuture = + CompletableFuture newFuture = findBroker(serviceNameResolver.resolveHost(), false, topicName, 0); newFutureCreated.setValue(newFuture); return newFuture; @@ -139,9 +138,9 @@ public CompletableFuture getPartitionedTopicMetadata(T } } - private CompletableFuture> findBroker(InetSocketAddress socketAddress, + private CompletableFuture findBroker(InetSocketAddress socketAddress, boolean authoritative, TopicName topicName, final int redirectCount) { - CompletableFuture> addressFuture = new CompletableFuture<>(); + CompletableFuture addressFuture = new CompletableFuture<>(); if (maxLookupRedirects > 0 && redirectCount > maxLookupRedirects) { addressFuture.completeExceptionally( @@ -159,7 +158,6 @@ private CompletableFuture> findBroker if (log.isDebugEnabled()) { log.debug("[{}] Lookup response exception: {}", topicName, t); } - addressFuture.completeExceptionally(t); } else { URI uri = null; @@ -198,10 +196,12 @@ private CompletableFuture> findBroker // (3) received correct broker to connect if (r.proxyThroughServiceUrl) { // Connect through proxy - addressFuture.complete(Pair.of(responseBrokerAddress, socketAddress)); + addressFuture.complete( + new LookupTopicResult(responseBrokerAddress, socketAddress, true)); } else { // Normal result with direct connection to broker - addressFuture.complete(Pair.of(responseBrokerAddress, responseBrokerAddress)); + addressFuture.complete( + new LookupTopicResult(responseBrokerAddress, responseBrokerAddress, false)); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 27ddd21249f86..75e84eeca3e6a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -323,8 +323,8 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { waitingLookupRequests.forEach(pair -> pair.getRight().getRight().completeExceptionally(e)); // Notify all attached producers/consumers so they have a chance to reconnect - producers.forEach((id, producer) -> producer.connectionClosed(this)); - consumers.forEach((id, consumer) -> consumer.connectionClosed(this)); + producers.forEach((id, producer) -> producer.connectionClosed(this, Optional.empty(), Optional.empty())); + consumers.forEach((id, consumer) -> consumer.connectionClosed(this, Optional.empty(), Optional.empty())); transactionMetaStoreHandlers.forEach((id, handler) -> handler.connectionClosed(this)); topicListWatchers.forEach((__, watcher) -> watcher.connectionClosed(this)); @@ -803,46 +803,72 @@ protected void handleError(CommandError error) { @Override protected void handleCloseProducer(CommandCloseProducer closeProducer) { final long producerId = closeProducer.getProducerId(); + log.info("[{}] Broker notification of closed producer: {}, assignedBrokerUrl: {}, assignedBrokerUrlTls: {}", + remoteAddress, producerId, + closeProducer.hasAssignedBrokerServiceUrl() ? closeProducer.getAssignedBrokerServiceUrl() : null, + closeProducer.hasAssignedBrokerServiceUrlTls() ? closeProducer.getAssignedBrokerServiceUrlTls() : null); ProducerImpl producer = producers.remove(producerId); if (producer != null) { - if (closeProducer.hasAssignedBrokerServiceUrl() || closeProducer.hasAssignedBrokerServiceUrlTls()) { - try { - final URI uri = new URI(producer.client.conf.isUseTls() - ? closeProducer.getAssignedBrokerServiceUrlTls() - : closeProducer.getAssignedBrokerServiceUrl()); - log.info("[{}] Broker notification of Closed producer: {}. Redirecting to {}.", - remoteAddress, closeProducer.getProducerId(), uri); - producer.getConnectionHandler().connectionClosed( - this, Optional.of(0L), Optional.of(uri)); - } catch (Throwable e) { - log.error("[{}] Invalid redirect url {}/{} for {}", remoteAddress, - closeProducer.hasAssignedBrokerServiceUrl() - ? closeProducer.getAssignedBrokerServiceUrl() : "", - closeProducer.hasAssignedBrokerServiceUrlTls() - ? closeProducer.getAssignedBrokerServiceUrlTls() : "", - closeProducer.getRequestId(), e); - producer.connectionClosed(this); - } - } else { - log.info("[{}] Broker notification of Closed producer: {}.", - remoteAddress, closeProducer.getProducerId()); - producer.connectionClosed(this); - } + String brokerServiceUrl = getBrokerServiceUrl(closeProducer, producer); + Optional hostUri = parseUri(brokerServiceUrl, + closeProducer.hasRequestId() ? closeProducer.getRequestId() : null); + Optional initialConnectionDelayMs = hostUri.map(__ -> 0L); + producer.connectionClosed(this, initialConnectionDelayMs, hostUri); } else { - log.warn("Producer with id {} not found while closing producer ", producerId); + log.warn("[{}] Producer with id {} not found while closing producer", remoteAddress, producerId); + } + } + + private static String getBrokerServiceUrl(CommandCloseProducer closeProducer, ProducerImpl producer) { + if (producer.getClient().getConfiguration().isUseTls()) { + if (closeProducer.hasAssignedBrokerServiceUrlTls()) { + return closeProducer.getAssignedBrokerServiceUrlTls(); + } + } else if (closeProducer.hasAssignedBrokerServiceUrl()) { + return closeProducer.getAssignedBrokerServiceUrl(); } + return null; } @Override protected void handleCloseConsumer(CommandCloseConsumer closeConsumer) { - log.info("[{}] Broker notification of Closed consumer: {}", remoteAddress, closeConsumer.getConsumerId()); final long consumerId = closeConsumer.getConsumerId(); + log.info("[{}] Broker notification of closed consumer: {}, assignedBrokerUrl: {}, assignedBrokerUrlTls: {}", + remoteAddress, consumerId, + closeConsumer.hasAssignedBrokerServiceUrl() ? closeConsumer.getAssignedBrokerServiceUrl() : null, + closeConsumer.hasAssignedBrokerServiceUrlTls() ? closeConsumer.getAssignedBrokerServiceUrlTls() : null); ConsumerImpl consumer = consumers.remove(consumerId); if (consumer != null) { - consumer.connectionClosed(this); + String brokerServiceUrl = getBrokerServiceUrl(closeConsumer, consumer); + Optional hostUri = parseUri(brokerServiceUrl, + closeConsumer.hasRequestId() ? closeConsumer.getRequestId() : null); + Optional initialConnectionDelayMs = hostUri.map(__ -> 0L); + consumer.connectionClosed(this, initialConnectionDelayMs, hostUri); } else { - log.warn("Consumer with id {} not found while closing consumer ", consumerId); + log.warn("[{}] Consumer with id {} not found while closing consumer", remoteAddress, consumerId); + } + } + + private static String getBrokerServiceUrl(CommandCloseConsumer closeConsumer, ConsumerImpl consumer) { + if (consumer.getClient().getConfiguration().isUseTls()) { + if (closeConsumer.hasAssignedBrokerServiceUrlTls()) { + return closeConsumer.getAssignedBrokerServiceUrlTls(); + } + } else if (closeConsumer.hasAssignedBrokerServiceUrl()) { + return closeConsumer.getAssignedBrokerServiceUrl(); + } + return null; + } + + private Optional parseUri(String url, Long requestId) { + try { + if (url != null) { + return Optional.of(new URI(url)); + } + } catch (URISyntaxException e) { + log.warn("[{}] Invalid redirect URL {}, requestId {}: ", remoteAddress, url, requestId, e); } + return Optional.empty(); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index 600dc17a1b09a..bee3121d7f689 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -47,6 +47,8 @@ public class ConnectionHandler { private final AtomicBoolean duringConnect = new AtomicBoolean(false); protected final int randomKeyForSelectConnection; + private boolean useProxy = false; + interface Connection { /** @@ -94,10 +96,13 @@ protected void grabCnx(Optional hostURI) { try { CompletableFuture cnxFuture; if (hostURI.isPresent()) { - InetSocketAddress address = InetSocketAddress.createUnresolved( - hostURI.get().getHost(), - hostURI.get().getPort()); - cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); + URI uri = hostURI.get(); + InetSocketAddress address = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); + if (useProxy) { + cnxFuture = state.client.getProxiedConnection(address, randomKeyForSelectConnection); + } else { + cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); + } } else if (state.redirectedClusterURI != null) { if (state.topic == null) { InetSocketAddress address = InetSocketAddress.createUnresolved(state.redirectedClusterURI.getHost(), @@ -112,7 +117,11 @@ protected void grabCnx(Optional hostURI) { } else if (state.topic == null) { cnxFuture = state.client.getConnectionToServiceUrl(); } else { - cnxFuture = state.client.getConnection(state.topic, randomKeyForSelectConnection); + cnxFuture = state.client.getConnection(state.topic, randomKeyForSelectConnection).thenApply( + connectionResult -> { + useProxy = connectionResult.getRight(); + return connectionResult.getLeft(); + }); } cnxFuture.thenCompose(cnx -> connection.connectionOpened(cnx)) .thenAccept(__ -> duringConnect.set(false)) @@ -182,11 +191,11 @@ public void connectionClosed(ClientCnx cnx, Optional initialConnectionDela } long delayMs = initialConnectionDelayMs.orElse(backoff.next()); state.setState(State.Connecting); - log.info("[{}] [{}] Closed connection {} -- Will try again in {} s", - state.topic, state.getHandlerName(), cnx.channel(), - delayMs / 1000.0); + log.info("[{}] [{}] Closed connection {} -- Will try again in {} s, hostUrl: {}", + state.topic, state.getHandlerName(), cnx.channel(), delayMs / 1000.0, hostUrl.orElse(null)); state.client.timer().newTimeout(timeout -> { - log.info("[{}] [{}] Reconnecting after timeout", state.topic, state.getHandlerName()); + log.info("[{}] [{}] Reconnecting after {} s timeout, hostUrl: {}", + state.topic, state.getHandlerName(), delayMs / 1000.0, hostUrl.orElse(null)); grabCnx(hostUrl); }, delayMs, TimeUnit.MILLISECONDS); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 6e27701fceaae..4f29c0aa76c94 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -711,8 +711,13 @@ public void negativeAcknowledge(Messages messages) { @Override public void unsubscribe() throws PulsarClientException { + unsubscribe(false); + } + + @Override + public void unsubscribe(boolean force) throws PulsarClientException { try { - unsubscribeAsync().get(); + unsubscribeAsync(force).get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw PulsarClientException.unwrap(e); @@ -722,7 +727,12 @@ public void unsubscribe() throws PulsarClientException { } @Override - public abstract CompletableFuture unsubscribeAsync(); + public CompletableFuture unsubscribeAsync() { + return unsubscribeAsync(false); + } + + @Override + public abstract CompletableFuture unsubscribeAsync(boolean force); @Override public void close() throws PulsarClientException { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index fbc2a8c285dd2..b43cd79959c00 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -35,6 +35,7 @@ import io.netty.util.Timeout; import io.netty.util.concurrent.FastThreadLocal; import java.io.IOException; +import java.net.URI; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -403,7 +404,7 @@ NegativeAcksTracker getNegativeAcksTracker() { } @Override - public CompletableFuture unsubscribeAsync() { + public CompletableFuture unsubscribeAsync(boolean force) { if (getState() == State.Closing || getState() == State.Closed) { return FutureUtil .failedFuture(new PulsarClientException.AlreadyClosedException("Consumer was already closed")); @@ -412,7 +413,7 @@ public CompletableFuture unsubscribeAsync() { if (isConnected()) { setState(State.Closing); long requestId = client.newRequestId(); - ByteBuf unsubscribe = Commands.newUnsubscribe(consumerId, requestId); + ByteBuf unsubscribe = Commands.newUnsubscribe(consumerId, requestId, force); ClientCnx cnx = cnx(); cnx.sendRequestWithId(unsubscribe, requestId).thenRun(() -> { closeConsumerTasks(); @@ -432,7 +433,7 @@ public CompletableFuture unsubscribeAsync() { }); } else { unsubscribeFuture.completeExceptionally( - new PulsarClientException( + new PulsarClientException.NotConnectedException( String.format("The client is not connected to the broker when unsubscribing the " + "subscription %s of the topic %s", subscription, topicName.toString()))); } @@ -888,7 +889,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // in case it was indeed created, otherwise it might prevent new create consumer operation, // since we are not necessarily closing the connection. long closeRequestId = client.newRequestId(); - ByteBuf cmd = Commands.newCloseConsumer(consumerId, closeRequestId); + ByteBuf cmd = Commands.newCloseConsumer(consumerId, closeRequestId, null, null); cnx.sendRequestWithId(cmd, closeRequestId); } @@ -1057,7 +1058,7 @@ public CompletableFuture closeAsync() { if (null == cnx) { cleanupAtClose(closeFuture, null); } else { - ByteBuf cmd = Commands.newCloseConsumer(consumerId, requestId); + ByteBuf cmd = Commands.newCloseConsumer(consumerId, requestId, null, null); cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> { final ChannelHandlerContext ctx = cnx.ctx(); boolean ignoreException = ctx == null || !ctx.channel().isActive(); @@ -2669,8 +2670,8 @@ void resetBackoff() { this.connectionHandler.resetBackoff(); } - void connectionClosed(ClientCnx cnx) { - this.connectionHandler.connectionClosed(cnx); + void connectionClosed(ClientCnx cnx, Optional initialConnectionDelayMs, Optional hostUrl) { + this.connectionHandler.connectionClosed(cnx, initialConnectionDelayMs, hostUrl); } public ClientCnx getClientCnx() { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index e33efabcc9e0e..33f90271826d2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -30,7 +30,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.NotFoundException; import org.apache.pulsar.client.api.SchemaSerializationException; @@ -81,7 +80,7 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException { */ @Override @SuppressWarnings("deprecation") - public CompletableFuture> getBroker(TopicName topicName) { + public CompletableFuture getBroker(TopicName topicName) { String basePath = topicName.isV2() ? BasePathV2 : BasePathV1; String path = basePath + topicName.getLookupName(); path = StringUtils.isBlank(listenerName) ? path : path + "?listenerName=" + Codec.encode(listenerName); @@ -101,7 +100,7 @@ public CompletableFuture> getBroker(T } InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); - return CompletableFuture.completedFuture(Pair.of(brokerAddress, brokerAddress)); + return CompletableFuture.completedFuture(new LookupTopicResult(brokerAddress, brokerAddress, false)); } catch (Exception e) { // Failed to parse url log.warn("[{}] Lookup Failed due to invalid url {}, {}", topicName, uri, e.getMessage()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index f0142f3612b4c..4d59d6591dbb8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -21,7 +21,6 @@ import java.net.InetSocketAddress; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; import org.apache.pulsar.common.lookup.GetTopicsResult; @@ -54,9 +53,10 @@ public interface LookupService extends AutoCloseable { * * @param topicName * topic-name - * @return a pair of addresses, representing the logical and physical address of the broker that serves given topic + * @return a {@link LookupTopicResult} representing the logical and physical address of the broker that serves the + * given topic, as well as proxying information. */ - CompletableFuture> getBroker(TopicName topicName); + CompletableFuture getBroker(TopicName topicName); /** * Returns {@link PartitionedTopicMetadata} for a given topic. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupTopicResult.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupTopicResult.java new file mode 100644 index 0000000000000..9730b5c1da58a --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupTopicResult.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.impl; + +import java.net.InetSocketAddress; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +@Getter +@Setter +@AllArgsConstructor +@ToString +public class LookupTopicResult { + private final InetSocketAddress logicalAddress; + private final InetSocketAddress physicalAddress; + private final boolean isUseProxy; +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 8a515a9f9b8d7..6ba3aaaaa4603 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -559,7 +559,7 @@ public void negativeAcknowledge(Message message) { } @Override - public CompletableFuture unsubscribeAsync() { + public CompletableFuture unsubscribeAsync(boolean force) { if (getState() == State.Closing || getState() == State.Closed) { return FutureUtil.failedFuture( new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); @@ -568,7 +568,7 @@ public CompletableFuture unsubscribeAsync() { CompletableFuture unsubscribeFuture = new CompletableFuture<>(); List> futureList = consumers.values().stream() - .map(ConsumerImpl::unsubscribeAsync).collect(Collectors.toList()); + .map(c -> c.unsubscribeAsync(force)).collect(Collectors.toList()); FutureUtil.waitForAll(futureList) .thenComposeAsync((r) -> { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index a17d4a06f02a6..2763da524cd58 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -41,6 +41,7 @@ import io.netty.util.TimerTask; import io.netty.util.concurrent.ScheduledFuture; import java.io.IOException; +import java.net.URI; import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; @@ -2372,8 +2373,8 @@ void resetBackoff() { this.connectionHandler.resetBackoff(); } - void connectionClosed(ClientCnx cnx) { - this.connectionHandler.connectionClosed(cnx); + void connectionClosed(ClientCnx cnx, Optional initialConnectionDelayMs, Optional hostUrl) { + this.connectionHandler.connectionClosed(cnx, initialConnectionDelayMs, hostUrl); } public ClientCnx getClientCnx() { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 50a3dbfc935b7..3b785fdaf2354 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicReference; import lombok.Builder; import lombok.Getter; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -946,10 +947,12 @@ public void updateTlsTrustStorePathAndPassword(String tlsTrustStorePath, String conf.setTlsTrustStorePassword(tlsTrustStorePassword); } - public CompletableFuture getConnection(final String topic, int randomKeyForSelectConnection) { - TopicName topicName = TopicName.get(topic); - return lookup.getBroker(topicName) - .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), randomKeyForSelectConnection)); + public CompletableFuture> getConnection(String topic, int randomKeyForSelectConnection) { + CompletableFuture lookupTopicResult = lookup.getBroker(TopicName.get(topic)); + CompletableFuture isUseProxy = lookupTopicResult.thenApply(LookupTopicResult::isUseProxy); + return lookupTopicResult.thenCompose(lookupResult -> getConnection(lookupResult.getLogicalAddress(), + lookupResult.getPhysicalAddress(), randomKeyForSelectConnection)). + thenCombine(isUseProxy, Pair::of); } /** @@ -957,15 +960,14 @@ public CompletableFuture getConnection(final String topic, int random */ @VisibleForTesting public CompletableFuture getConnection(final String topic) { - TopicName topicName = TopicName.get(topic); - return lookup.getBroker(topicName) - .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); + return getConnection(topic, cnxPool.genRandomKeyToSelectCon()).thenApply(Pair::getLeft); } public CompletableFuture getConnection(final String topic, final String url) { TopicName topicName = TopicName.get(topic); return getLookup(url).getBroker(topicName) - .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); + .thenCompose(lookupResult -> getConnection(lookupResult.getLogicalAddress(), + lookupResult.getPhysicalAddress(), cnxPool.genRandomKeyToSelectCon())); } public LookupService getLookup(String serviceUrl) { @@ -988,6 +990,15 @@ public CompletableFuture getConnectionToServiceUrl() { return getConnection(address, address, cnxPool.genRandomKeyToSelectCon()); } + public CompletableFuture getProxiedConnection(final InetSocketAddress logicalAddress, + final int randomKeyForSelectConnection) { + if (!(lookup instanceof BinaryProtoLookupService)) { + return FutureUtil.failedFuture(new PulsarClientException.InvalidServiceURL( + "Cannot proxy connection through HTTP service URL", null)); + } + return getConnection(logicalAddress, lookup.resolveHost(), randomKeyForSelectConnection); + } + public CompletableFuture getConnection(final InetSocketAddress logicalAddress, final InetSocketAddress physicalAddress, final int randomKeyForSelectConnection) { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java index 0254cf8d44cea..87188255b20b8 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java @@ -27,16 +27,12 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import io.netty.buffer.ByteBuf; - import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; - -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException.LookupException; import org.apache.pulsar.client.impl.BinaryProtoLookupService.LookupDataResult; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; @@ -80,11 +76,12 @@ public void setup() throws Exception { @Test(invocationTimeOut = 3000) public void maxLookupRedirectsTest1() throws Exception { - Pair addressPair = lookup.getBroker(topicName).get(); - assertEquals(addressPair.getLeft(), InetSocketAddress + LookupTopicResult lookupResult = lookup.getBroker(topicName).get(); + assertEquals(lookupResult.getLogicalAddress(), InetSocketAddress .createUnresolved("broker2.pulsar.apache.org" ,6650)); - assertEquals(addressPair.getRight(), InetSocketAddress + assertEquals(lookupResult.getPhysicalAddress(), InetSocketAddress .createUnresolved("broker2.pulsar.apache.org" ,6650)); + assertEquals(lookupResult.isUseProxy(), false); } @Test(invocationTimeOut = 3000) @@ -93,11 +90,12 @@ public void maxLookupRedirectsTest2() throws Exception { field.setAccessible(true); field.set(lookup, 2); - Pair addressPair = lookup.getBroker(topicName).get(); - assertEquals(addressPair.getLeft(), InetSocketAddress + LookupTopicResult lookupResult = lookup.getBroker(topicName).get(); + assertEquals(lookupResult.getLogicalAddress(), InetSocketAddress .createUnresolved("broker2.pulsar.apache.org" ,6650)); - assertEquals(addressPair.getRight(), InetSocketAddress + assertEquals(lookupResult.getPhysicalAddress(), InetSocketAddress .createUnresolved("broker2.pulsar.apache.org" ,6650)); + assertEquals(lookupResult.isUseProxy(), false); } @Test(invocationTimeOut = 3000) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java index 22220805814f5..4f657da82b289 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java @@ -33,6 +33,7 @@ import io.netty.channel.EventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; import java.lang.reflect.Field; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -278,13 +279,19 @@ public void testHandleCloseConsumer() { ClientCnx cnx = new ClientCnx(conf, eventLoop); long consumerId = 1; - cnx.registerConsumer(consumerId, mock(ConsumerImpl.class)); + PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + when(pulsarClient.getConfiguration()).thenReturn(conf); + ConsumerImpl consumer = mock(ConsumerImpl.class); + when(consumer.getClient()).thenReturn(pulsarClient); + cnx.registerConsumer(consumerId, consumer); assertEquals(cnx.consumers.size(), 1); - CommandCloseConsumer closeConsumer = new CommandCloseConsumer().setConsumerId(consumerId); + CommandCloseConsumer closeConsumer = new CommandCloseConsumer().setConsumerId(consumerId).setRequestId(1); cnx.handleCloseConsumer(closeConsumer); assertEquals(cnx.consumers.size(), 0); + verify(consumer).connectionClosed(cnx, Optional.empty(), Optional.empty()); + eventLoop.shutdownGracefully(); } @@ -296,13 +303,19 @@ public void testHandleCloseProducer() { ClientCnx cnx = new ClientCnx(conf, eventLoop); long producerId = 1; - cnx.registerProducer(producerId, mock(ProducerImpl.class)); + PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + when(pulsarClient.getConfiguration()).thenReturn(conf); + ProducerImpl producer = mock(ProducerImpl.class); + when(producer.getClient()).thenReturn(pulsarClient); + cnx.registerProducer(producerId, producer); assertEquals(cnx.producers.size(), 1); - CommandCloseProducer closeProducerCmd = new CommandCloseProducer().setProducerId(producerId); + CommandCloseProducer closeProducerCmd = new CommandCloseProducer().setProducerId(producerId).setRequestId(1); cnx.handleCloseProducer(closeProducerCmd); assertEquals(cnx.producers.size(), 0); + verify(producer).connectionClosed(cnx, Optional.empty(), Optional.empty()); + eventLoop.shutdownGracefully(); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java index 738d969ac7449..915c3dcc05a7e 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java @@ -34,6 +34,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.util.ExecutorProvider; import org.mockito.Mockito; @@ -82,7 +83,7 @@ static PulsarClientImpl mockClientCnx(PulsarClientImpl clientMock) { when(clientMock.getConnection(any())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); when(clientMock.getConnection(anyString())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); when(clientMock.getConnection(anyString(), anyInt())) - .thenReturn(CompletableFuture.completedFuture(clientCnxMock)); + .thenReturn(CompletableFuture.completedFuture(Pair.of(clientCnxMock, false))); when(clientMock.getConnection(any(), any(), anyInt())) .thenReturn(CompletableFuture.completedFuture(clientCnxMock)); ConnectionPool connectionPoolMock = mock(ConnectionPool.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index 9a4cfce0cc3da..c96443c1e2f9f 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -50,7 +50,6 @@ import java.util.concurrent.ThreadFactory; import java.util.regex.Pattern; import lombok.Cleanup; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; @@ -110,8 +109,8 @@ public void testConsumerIsClosed() throws Exception { when(lookup.getPartitionedTopicMetadata(any(TopicName.class))) .thenReturn(CompletableFuture.completedFuture(new PartitionedTopicMetadata())); when(lookup.getBroker(any())) - .thenReturn(CompletableFuture.completedFuture( - Pair.of(mock(InetSocketAddress.class), mock(InetSocketAddress.class)))); + .thenReturn(CompletableFuture.completedFuture(new LookupTopicResult( + mock(InetSocketAddress.class), mock(InetSocketAddress.class), false))); ConnectionPool pool = mock(ConnectionPool.class); ClientCnx cnx = mock(ClientCnx.class); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index 1b39448fbe770..dd75770b5688d 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -21,6 +21,7 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl.TopicsChangedListener; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.api.proto.BaseCommand; @@ -63,8 +64,8 @@ public void setup() { Timer timer = new HashedWheelTimer(); when(client.timer()).thenReturn(timer); String topic = "persistent://tenant/ns/topic\\d+"; - when(client.getConnection(topic)).thenReturn(clientCnxFuture); - when(client.getConnection(topic, 0)).thenReturn(clientCnxFuture); + when(client.getConnection(topic, 0)). + thenReturn(clientCnxFuture.thenApply(clientCnx -> Pair.of(clientCnx, false))); when(client.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); when(connectionPool.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); watcherFuture = new CompletableFuture<>(); @@ -120,6 +121,4 @@ public void testWatcherCallsListenerOnUpdate() { watcher.handleCommandWatchTopicUpdate(update); verify(listener).onTopicsAdded(Collections.singletonList("persistent://tenant/ns/topic12")); } - - } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index ff116d2406b40..34d47e2836bb2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -60,6 +60,7 @@ import org.apache.pulsar.common.api.proto.CommandAddSubscriptionToTxn; import org.apache.pulsar.common.api.proto.CommandAddSubscriptionToTxnResponse; import org.apache.pulsar.common.api.proto.CommandAuthChallenge; +import org.apache.pulsar.common.api.proto.CommandCloseConsumer; import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.api.proto.CommandConnect; import org.apache.pulsar.common.api.proto.CommandConnected; @@ -697,11 +698,12 @@ private static KeySharedMode convertKeySharedMode(org.apache.pulsar.client.api.K } } - public static ByteBuf newUnsubscribe(long consumerId, long requestId) { + public static ByteBuf newUnsubscribe(long consumerId, long requestId, boolean force) { BaseCommand cmd = localCmd(Type.UNSUBSCRIBE); cmd.setUnsubscribe() .setConsumerId(consumerId) - .setRequestId(requestId); + .setRequestId(requestId) + .setForce(force); return serializeWithSize(cmd); } @@ -737,11 +739,21 @@ public static ByteBuf newSeek(long consumerId, long requestId, long timestamp) { return serializeWithSize(cmd); } - public static ByteBuf newCloseConsumer(long consumerId, long requestId) { + public static ByteBuf newCloseConsumer( + long consumerId, long requestId, String assignedBrokerUrl, String assignedBrokerUrlTls) { BaseCommand cmd = localCmd(Type.CLOSE_CONSUMER); - cmd.setCloseConsumer() + CommandCloseConsumer commandCloseConsumer = cmd.setCloseConsumer() .setConsumerId(consumerId) .setRequestId(requestId); + + if (assignedBrokerUrl != null) { + commandCloseConsumer.setAssignedBrokerServiceUrl(assignedBrokerUrl); + } + + if (assignedBrokerUrlTls != null) { + commandCloseConsumer.setAssignedBrokerServiceUrlTls(assignedBrokerUrlTls); + } + return serializeWithSize(cmd); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java index 2f0a7884dde35..b4e15f8cd1d75 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java @@ -19,36 +19,63 @@ package org.apache.pulsar.common.protocol; import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.ProtocolDetectionResult; import io.netty.handler.codec.ProtocolDetectionState; import io.netty.handler.codec.haproxy.HAProxyMessageDecoder; import io.netty.handler.codec.haproxy.HAProxyProtocolVersion; +import lombok.extern.slf4j.Slf4j; /** * Decoder that added whether a new connection is prefixed with the ProxyProtocol. * More about the ProxyProtocol see: http://www.haproxy.org/download/1.8/doc/proxy-protocol.txt. */ +@Slf4j public class OptionalProxyProtocolDecoder extends ChannelInboundHandlerAdapter { public static final String NAME = "optional-proxy-protocol-decoder"; + public static final int MIN_BYTES_SIZE_TO_DETECT_PROTOCOL = 12; + + private CompositeByteBuf cumulatedByteBuf; + @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { if (msg instanceof ByteBuf) { - ProtocolDetectionResult result = - HAProxyMessageDecoder.detectProtocol((ByteBuf) msg); - // should accumulate data if need more data to detect the protocol + // Combine cumulated buffers. + ByteBuf buf = (ByteBuf) msg; + if (cumulatedByteBuf != null) { + buf = cumulatedByteBuf.addComponent(true, buf); + } + + ProtocolDetectionResult result = HAProxyMessageDecoder.detectProtocol(buf); if (result.state() == ProtocolDetectionState.NEEDS_MORE_DATA) { + // Accumulate data if need more data to detect the protocol. + if (cumulatedByteBuf == null) { + cumulatedByteBuf = new CompositeByteBuf(ctx.alloc(), false, MIN_BYTES_SIZE_TO_DETECT_PROTOCOL, buf); + } return; } + cumulatedByteBuf = null; if (result.state() == ProtocolDetectionState.DETECTED) { ctx.pipeline().addAfter(NAME, null, new HAProxyMessageDecoder()); - ctx.pipeline().remove(this); } + ctx.pipeline().remove(this); + super.channelRead(ctx, buf); + } else { + super.channelRead(ctx, msg); + } + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + if (cumulatedByteBuf != null) { + log.info("Release cumulated byte buffer when channel inactive."); + cumulatedByteBuf = null; } - super.channelRead(ctx, msg); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java deleted file mode 100644 index 4ecb29b2462cc..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util; - -import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; -import com.google.common.base.MoreObjects; -import io.netty.util.concurrent.DefaultThreadFactory; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; -import lombok.Builder; - -/** - * A Rate Limiter that distributes permits at a configurable rate. Each {@link #acquire()} blocks if necessary until a - * permit is available, and then takes it. Each {@link #tryAcquire()} tries to acquire permits from available permits, - * it returns true if it succeed else returns false. Rate limiter release configured permits at every configured rate - * time, so, on next ticket new fresh permits will be available. - * - *

For example: if RateLimiter is configured to release 10 permits at every 1 second then RateLimiter will allow to - * acquire 10 permits at any time with in that 1 second. - * - *

Comparison with other RateLimiter such as {@link com.google.common.util.concurrent.RateLimiter} - *

    - *
  • Per second rate-limiting: Per second rate-limiting not satisfied by Guava-RateLimiter
  • - *
  • Guava RateLimiter: For X permits: it releases X/1000 permits every msec. therefore, - * for permits=2/sec => it release 1st permit on first 500msec and 2nd permit on next 500ms. therefore, - * if 2 request comes with in 500msec duration then 2nd request fails to acquire permit - * though we have configured 2 permits/second.
  • - *
  • RateLimiter: it releases X permits every second. so, in above usecase: - * if 2 requests comes at the same time then both will acquire the permit.
  • - *
  • Faster: RateLimiter is light-weight and faster than Guava-RateLimiter
  • - *
- */ -public class RateLimiter implements AutoCloseable{ - private final ScheduledExecutorService executorService; - private long rateTime; - private TimeUnit timeUnit; - private final boolean externalExecutor; - private ScheduledFuture renewTask; - private volatile long permits; - private volatile long acquiredPermits; - private boolean isClosed; - // permitUpdate helps to update permit-rate at runtime - private Supplier permitUpdater; - private RateLimitFunction rateLimitFunction; - private boolean isDispatchOrPrecisePublishRateLimiter; - - @Builder - RateLimiter(final ScheduledExecutorService scheduledExecutorService, final long permits, final long rateTime, - final TimeUnit timeUnit, Supplier permitUpdater, boolean isDispatchOrPrecisePublishRateLimiter, - RateLimitFunction rateLimitFunction) { - checkArgument(permits > 0, "rate must be > 0"); - checkArgument(rateTime > 0, "Renew permit time must be > 0"); - - this.rateTime = rateTime; - this.timeUnit = timeUnit; - this.permits = permits; - this.permitUpdater = permitUpdater; - this.isDispatchOrPrecisePublishRateLimiter = isDispatchOrPrecisePublishRateLimiter; - - if (scheduledExecutorService != null) { - this.executorService = scheduledExecutorService; - this.externalExecutor = true; - } else { - final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, - new DefaultThreadFactory("pulsar-rate-limiter")); - executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false); - executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); - this.executorService = executor; - this.externalExecutor = false; - } - - this.rateLimitFunction = rateLimitFunction; - - } - - // default values for Lombok generated builder class - public static class RateLimiterBuilder { - private long rateTime = 1; - private TimeUnit timeUnit = TimeUnit.SECONDS; - } - - @Override - public synchronized void close() { - if (!isClosed) { - if (!externalExecutor) { - executorService.shutdownNow(); - } - if (renewTask != null) { - renewTask.cancel(false); - } - isClosed = true; - // If there is a ratelimit function registered, invoke it to unblock. - if (rateLimitFunction != null) { - rateLimitFunction.apply(); - } - } - } - - public synchronized boolean isClosed() { - return isClosed; - } - - /** - * Acquires the given number of permits from this {@code RateLimiter}, blocking until the request be granted. - * - *

This method is equivalent to {@code acquire(1)}. - */ - public synchronized void acquire() throws InterruptedException { - acquire(1); - } - - /** - * Acquires the given number of permits from this {@code RateLimiter}, blocking until the request be granted. - * - * @param acquirePermit - * the number of permits to acquire - */ - public synchronized void acquire(long acquirePermit) throws InterruptedException { - checkArgument(!isClosed(), "Rate limiter is already shutdown"); - checkArgument(acquirePermit <= this.permits, - "acquiring permits must be less or equal than initialized rate =" + this.permits); - - // lazy init and start task only once application start using it - if (renewTask == null) { - renewTask = createTask(); - } - - boolean canAcquire = false; - do { - canAcquire = acquirePermit < 0 || acquiredPermits < this.permits; - if (!canAcquire) { - wait(); - } else { - acquiredPermits += acquirePermit; - } - } while (!canAcquire); - } - - /** - * Acquires permits from this {@link RateLimiter} if it can be acquired immediately without delay. - * - *

This method is equivalent to {@code tryAcquire(1)}. - * - * @return {@code true} if the permits were acquired, {@code false} otherwise - */ - public synchronized boolean tryAcquire() { - return tryAcquire(1); - } - - /** - * Acquires permits from this {@link RateLimiter} if it can be acquired immediately without delay. - * - * @param acquirePermit - * the number of permits to acquire - * @return {@code true} if the permits were acquired, {@code false} otherwise - */ - public synchronized boolean tryAcquire(long acquirePermit) { - checkArgument(!isClosed(), "Rate limiter is already shutdown"); - // lazy init and start task only once application start using it - if (renewTask == null) { - renewTask = createTask(); - } - - boolean canAcquire = acquirePermit < 0 || acquiredPermits < this.permits; - if (isDispatchOrPrecisePublishRateLimiter) { - // for dispatch rate limiter just add acquirePermit - acquiredPermits += acquirePermit; - - // we want to back-pressure from the current state of the rateLimiter therefore we should check if there - // are any available premits again - canAcquire = acquirePermit < 0 || acquiredPermits < this.permits; - } else { - // acquired-permits can't be larger than the rate - if (acquirePermit + acquiredPermits > this.permits) { - return false; - } - - if (canAcquire) { - acquiredPermits += acquirePermit; - } - } - - return canAcquire; - } - - /** - * Return available permits for this {@link RateLimiter}. - * - * @return returns 0 if permits is not available - */ - public long getAvailablePermits() { - return Math.max(0, this.permits - this.acquiredPermits); - } - - /** - * Resets new rate by configuring new value for permits per configured rate-period. - * - * @param permits - */ - public synchronized void setRate(long permits) { - this.permits = permits; - } - - /** - * Resets new rate with new permits and rate-time. - * - * @param permits - * @param rateTime - * @param timeUnit - * @param permitUpdaterByte - */ - public synchronized void setRate(long permits, long rateTime, TimeUnit timeUnit, Supplier permitUpdaterByte) { - if (renewTask != null) { - renewTask.cancel(false); - } - this.permits = permits; - this.rateTime = rateTime; - this.timeUnit = timeUnit; - this.permitUpdater = permitUpdaterByte; - this.renewTask = createTask(); - } - - /** - * Returns configured permit rate per pre-configured rate-period. - * - * @return rate - */ - public synchronized long getRate() { - return this.permits; - } - - public synchronized long getRateTime() { - return this.rateTime; - } - - public synchronized TimeUnit getRateTimeUnit() { - return this.timeUnit; - } - - protected ScheduledFuture createTask() { - return executorService.scheduleAtFixedRate(catchingAndLoggingThrowables(this::renew), this.rateTime, - this.rateTime, this.timeUnit); - } - - synchronized void renew() { - acquiredPermits = isDispatchOrPrecisePublishRateLimiter ? Math.max(0, acquiredPermits - permits) : 0; - if (permitUpdater != null) { - long newPermitRate = permitUpdater.get(); - if (newPermitRate > 0) { - setRate(newPermitRate); - } - } - // release the back-pressure by applying the rateLimitFunction only when there are available permits - if (rateLimitFunction != null && this.getAvailablePermits() > 0) { - rateLimitFunction.apply(); - } - notifyAll(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("rateTime", rateTime).add("permits", permits) - .add("acquiredPermits", acquiredPermits).toString(); - } - -} diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index 2c350aaf8a10e..387e4e3ff679d 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -607,6 +607,7 @@ message CommandFlow { message CommandUnsubscribe { required uint64 consumer_id = 1; required uint64 request_id = 2; + optional bool force = 3 [default = false]; } // Reset an existing consumer to a particular message id @@ -648,6 +649,8 @@ message CommandCloseProducer { message CommandCloseConsumer { required uint64 consumer_id = 1; required uint64 request_id = 2; + optional string assignedBrokerServiceUrl = 3; + optional string assignedBrokerServiceUrlTls = 4; } message CommandRedeliverUnacknowledgedMessages { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/RateLimiterTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/RateLimiterTest.java deleted file mode 100644 index 3738027c63549..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/RateLimiterTest.java +++ /dev/null @@ -1,248 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import lombok.Cleanup; -import org.testng.annotations.Test; - -public class RateLimiterTest { - - @Test - public void testInvalidRenewTime() { - try { - RateLimiter.builder().permits(0).rateTime(100).timeUnit(TimeUnit.SECONDS).build(); - fail("should have thrown exception: invalid rate, must be > 0"); - } catch (IllegalArgumentException ie) { - // Ok - } - - try { - RateLimiter.builder().permits(10).rateTime(0).timeUnit(TimeUnit.SECONDS).build(); - fail("should have thrown exception: invalid rateTime, must be > 0"); - } catch (IllegalArgumentException ie) { - // Ok - } - } - - @Test - public void testClose() throws Exception { - RateLimiter rate = RateLimiter.builder().permits(1).rateTime(1000).timeUnit(TimeUnit.MILLISECONDS).build(); - assertFalse(rate.isClosed()); - rate.close(); - assertTrue(rate.isClosed()); - try { - rate.acquire(); - fail("should have failed, executor is already closed"); - } catch (IllegalArgumentException e) { - // ok - } - } - - @Test - public void testAcquireBlock() throws Exception { - final long rateTimeMSec = 1000; - RateLimiter rate = RateLimiter.builder().permits(1).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - rate.acquire(); - assertEquals(rate.getAvailablePermits(), 0); - long start = System.currentTimeMillis(); - rate.acquire(); - long end = System.currentTimeMillis(); - // no permits are available: need to wait on acquire - assertTrue((end - start) > rateTimeMSec / 2); - rate.close(); - } - - @Test - public void testAcquire() throws Exception { - final long rateTimeMSec = 1000; - final int permits = 100; - RateLimiter rate = RateLimiter.builder().permits(permits).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - long start = System.currentTimeMillis(); - for (int i = 0; i < permits; i++) { - rate.acquire(); - } - long end = System.currentTimeMillis(); - assertTrue((end - start) < rateTimeMSec); - assertEquals(rate.getAvailablePermits(), 0); - rate.close(); - } - - @Test - public void testMultipleAcquire() throws Exception { - final long rateTimeMSec = 1000; - final int permits = 100; - final int acquirePermits = 50; - RateLimiter rate = RateLimiter.builder().permits(permits).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - long start = System.currentTimeMillis(); - for (int i = 0; i < permits / acquirePermits; i++) { - rate.acquire(acquirePermits); - } - long end = System.currentTimeMillis(); - assertTrue((end - start) < rateTimeMSec); - assertEquals(rate.getAvailablePermits(), 0); - rate.close(); - } - - @Test - public void testTryAcquireNoPermits() { - final long rateTimeMSec = 1000; - RateLimiter rate = RateLimiter.builder().permits(1).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - assertTrue(rate.tryAcquire()); - assertFalse(rate.tryAcquire()); - assertEquals(rate.getAvailablePermits(), 0); - rate.close(); - } - - @Test - public void testTryAcquire() { - final long rateTimeMSec = 1000; - final int permits = 100; - RateLimiter rate = RateLimiter.builder().permits(permits).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - for (int i = 0; i < permits; i++) { - rate.tryAcquire(); - } - assertEquals(rate.getAvailablePermits(), 0); - rate.close(); - } - - @Test - public void testTryAcquireMoreThanPermits() { - final long rateTimeMSec = 1000; - RateLimiter rate = RateLimiter.builder().permits(3).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - assertTrue(rate.tryAcquire(2)); - assertEquals(rate.getAvailablePermits(), 1); - - //try to acquire failed, not decrease availablePermits. - assertFalse(rate.tryAcquire(2)); - assertEquals(rate.getAvailablePermits(), 1); - - assertTrue(rate.tryAcquire(1)); - assertEquals(rate.getAvailablePermits(), 0); - - rate.close(); - } - - @Test - public void testMultipleTryAcquire() { - final long rateTimeMSec = 1000; - final int permits = 100; - final int acquirePermits = 50; - RateLimiter rate = RateLimiter.builder().permits(permits).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - for (int i = 0; i < permits / acquirePermits; i++) { - rate.tryAcquire(acquirePermits); - } - assertEquals(rate.getAvailablePermits(), 0); - rate.close(); - } - - @Test - public void testResetRate() throws Exception { - final long rateTimeMSec = 1000; - final int permits = 100; - RateLimiter rate = RateLimiter.builder().permits(permits).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .build(); - rate.tryAcquire(permits); - assertEquals(rate.getAvailablePermits(), 0); - // check after a rate-time: permits must be renewed - Thread.sleep(rateTimeMSec * 2); - assertEquals(rate.getAvailablePermits(), permits); - - // change rate-time from 1sec to 5sec - rate.setRate(permits, 5 * rateTimeMSec, TimeUnit.MILLISECONDS, null); - assertEquals(rate.getAvailablePermits(), 100); - assertTrue(rate.tryAcquire(permits)); - assertEquals(rate.getAvailablePermits(), 0); - // check after a rate-time: permits can't be renewed - Thread.sleep(rateTimeMSec); - assertEquals(rate.getAvailablePermits(), 0); - - rate.close(); - } - - @Test - public void testDispatchRate() throws Exception { - final long rateTimeMSec = 1000; - final int permits = 100; - RateLimiter rate = RateLimiter.builder().permits(permits).rateTime(rateTimeMSec).timeUnit(TimeUnit.MILLISECONDS) - .isDispatchOrPrecisePublishRateLimiter(true) - .build(); - rate.tryAcquire(100); - rate.tryAcquire(100); - rate.tryAcquire(100); - assertEquals(rate.getAvailablePermits(), 0); - - Thread.sleep(rateTimeMSec * 2); - // check after two rate-time: acquiredPermits is 100 - assertEquals(rate.getAvailablePermits(), 0); - - Thread.sleep(rateTimeMSec); - // check after three rate-time: acquiredPermits is 0 - assertTrue(rate.getAvailablePermits() > 0); - - rate.close(); - } - - @Test - public void testRateLimiterWithPermitUpdater() throws Exception { - long permits = 10; - long rateTime = 1; - long newUpdatedRateLimit = 100L; - Supplier permitUpdater = () -> newUpdatedRateLimit; - @Cleanup - RateLimiter limiter = RateLimiter.builder().permits(permits).rateTime(1).timeUnit(TimeUnit.SECONDS) - .permitUpdater(permitUpdater) - .build(); - limiter.acquire(); - Thread.sleep(rateTime * 3 * 1000); - assertEquals(limiter.getAvailablePermits(), newUpdatedRateLimit); - } - - @Test - public void testRateLimiterWithFunction() { - final AtomicInteger atomicInteger = new AtomicInteger(0); - long permits = 10; - long rateTime = 1; - int reNewTime = 3; - RateLimitFunction rateLimitFunction = atomicInteger::incrementAndGet; - @Cleanup - RateLimiter rateLimiter = RateLimiter.builder().permits(permits).rateTime(rateTime).timeUnit(TimeUnit.SECONDS) - .rateLimitFunction(rateLimitFunction) - .build(); - for (int i = 0; i < reNewTime; i++) { - rateLimiter.renew(); - } - assertEquals(reNewTime, atomicInteger.get()); - } - -} diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java index b3850cbb53dac..21f125d349738 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java @@ -390,7 +390,7 @@ private void setupStateStore() throws Exception { stateStoreProvider = getStateStoreProvider(); Map stateStoreProviderConfig = new HashMap<>(); stateStoreProviderConfig.put(BKStateStoreProviderImpl.STATE_STORAGE_SERVICE_URL, stateStorageServiceUrl); - stateStoreProvider.init(stateStoreProviderConfig, instanceConfig.getFunctionDetails()); + stateStoreProvider.init(stateStoreProviderConfig); StateStore store = stateStoreProvider.getStateStore( instanceConfig.getFunctionDetails().getTenant(), diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/BKStateStoreProviderImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/BKStateStoreProviderImpl.java index dbd0c8d2a0254..5faab27b341d3 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/BKStateStoreProviderImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/BKStateStoreProviderImpl.java @@ -45,7 +45,6 @@ import org.apache.bookkeeper.stream.proto.StorageType; import org.apache.bookkeeper.stream.proto.StreamConfiguration; import org.apache.pulsar.functions.api.StateStore; -import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.apache.pulsar.functions.utils.FunctionCommon; /** @@ -58,7 +57,7 @@ public class BKStateStoreProviderImpl implements StateStoreProvider { private Map clients; @Override - public void init(Map config, FunctionDetails functionDetails) throws Exception { + public void init(Map config) throws Exception { stateStorageServiceUrl = (String) config.get(STATE_STORAGE_SERVICE_URL); clients = new HashMap<>(); } @@ -190,6 +189,29 @@ public T getStateStore(String tenant, String namespace, S return (T) new BKStateStoreImpl(tenant, namespace, name, table); } + @Override + public void cleanUp(String tenant, String namespace, String name) throws Exception { + StorageAdminClient storageAdminClient = new SimpleStorageAdminClientImpl( + StorageClientSettings.newBuilder().serviceUri(stateStorageServiceUrl).build(), + ClientResources.create().scheduler()); + String tableNs = FunctionCommon.getStateNamespace(tenant, namespace); + storageAdminClient.deleteStream(tableNs, name).whenComplete((res, throwable) -> { + if ((throwable == null && res) + || ((throwable instanceof NamespaceNotFoundException + || throwable instanceof StreamNotFoundException))) { + log.info("{}/{} table deleted successfully", tableNs, name); + } else { + if (throwable != null) { + log.error("{}/{} table deletion failed {} but moving on", tableNs, name, throwable); + } else { + log.error("{}/{} table deletion failed but moving on", tableNs, name); + } + } + }); + storageAdminClient.close(); + } + + @Override public void close() { clients.forEach((name, client) -> client.closeAsync() diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/PulsarMetadataStateStoreProviderImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/PulsarMetadataStateStoreProviderImpl.java index 0674398d1acda..7b9807b2a7816 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/PulsarMetadataStateStoreProviderImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/PulsarMetadataStateStoreProviderImpl.java @@ -20,7 +20,6 @@ import java.util.Map; import lombok.SneakyThrows; -import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreFactory; @@ -38,7 +37,7 @@ public class PulsarMetadataStateStoreProviderImpl implements StateStoreProvider private boolean shouldCloseStore; @Override - public void init(Map config, Function.FunctionDetails functionDetails) throws Exception { + public void init(Map config) throws Exception { prefix = (String) config.getOrDefault(METADATA_PREFIX, METADATA_DEFAULT_PREFIX); @@ -58,6 +57,13 @@ public DefaultStateStore getStateStore(String tenant, String namespace, String n return new PulsarMetadataStateStoreImpl(store, prefix, tenant, namespace, name); } + @Override + public void cleanUp(String tenant, String namespace, String name) throws Exception { + String fqsn = tenant + '/' + namespace + '/' + name; + String prefixPath = prefix + '/' + fqsn + '/'; + store.deleteRecursive(prefixPath); + } + @SneakyThrows @Override public void close() { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/StateStoreProvider.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/StateStoreProvider.java index 1602d8f5ba367..4088888e4a5d4 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/StateStoreProvider.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/StateStoreProvider.java @@ -51,8 +51,17 @@ public void close() { * @param functionDetails the function details. * @throws Exception when failed to init the state store provider. */ + @Deprecated default void init(Map config, FunctionDetails functionDetails) throws Exception {} + /** + * Initialize the state store provider. + * + * @param config the config to init the state store provider. + * @throws Exception when failed to init the state store provider. + */ + default void init(Map config) throws Exception {} + /** * Get the state store with the provided store name. * @@ -67,6 +76,16 @@ default void init(Map config, FunctionDetails functionDetails) t */ T getStateStore(String tenant, String namespace, String name) throws Exception; + /** + * Clean up the state store with the provided store name. + * + * @param tenant the tenant that owns this state store + * @param namespace the namespace that owns this state store + * @param name the state store name + * @throws Exception when failed to clean up the state store provider. + */ + default void cleanUp(String tenant, String namespace, String name) throws Exception {} + @Override void close(); } diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java index 1bb46da947224..8737c8a4fa913 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java @@ -19,16 +19,15 @@ package org.apache.pulsar.functions.instance.stats; import com.google.common.collect.EvictingQueue; +import com.google.common.util.concurrent.RateLimiter; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import io.prometheus.client.Summary; import java.util.Arrays; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.common.util.RateLimiter; import org.apache.pulsar.functions.proto.InstanceCommunication; /** @@ -262,18 +261,8 @@ public FunctionStatsManager(FunctionCollectorRegistry collectorRegistry, .help("Exception from sink.") .create()); - userExceptionRateLimiter = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(5) - .rateTime(1) - .timeUnit(TimeUnit.MINUTES) - .build(); - sysExceptionRateLimiter = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(5) - .rateTime(1) - .timeUnit(TimeUnit.MINUTES) - .build(); + userExceptionRateLimiter = RateLimiter.create(5.0d / 60.0d); + sysExceptionRateLimiter = RateLimiter.create(5.0d / 60.0d); } public void addUserException(Throwable ex) { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java index 779a56cdf61be..c515ce6bc872c 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java @@ -19,13 +19,12 @@ package org.apache.pulsar.functions.instance.stats; import com.google.common.collect.EvictingQueue; +import com.google.common.util.concurrent.RateLimiter; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import java.util.Arrays; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import lombok.Getter; -import org.apache.pulsar.common.util.RateLimiter; import org.apache.pulsar.functions.proto.InstanceCommunication; public class SinkStatsManager extends ComponentStatsManager { @@ -196,18 +195,8 @@ public SinkStatsManager(FunctionCollectorRegistry collectorRegistry, String[] me .help("Exception from sink.") .create()); - sysExceptionRateLimiter = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(5) - .rateTime(1) - .timeUnit(TimeUnit.MINUTES) - .build(); - sinkExceptionRateLimiter = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(5) - .rateTime(1) - .timeUnit(TimeUnit.MINUTES) - .build(); + sysExceptionRateLimiter = RateLimiter.create(5.0d / 60.0d); + sinkExceptionRateLimiter = RateLimiter.create(5.0d / 60.0d); } @Override diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java index 4470310c6c3f7..1f7e159c4dcb5 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java @@ -19,13 +19,12 @@ package org.apache.pulsar.functions.instance.stats; import com.google.common.collect.EvictingQueue; +import com.google.common.util.concurrent.RateLimiter; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import java.util.Arrays; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import lombok.Getter; -import org.apache.pulsar.common.util.RateLimiter; import org.apache.pulsar.functions.proto.InstanceCommunication; public class SourceStatsManager extends ComponentStatsManager { @@ -196,18 +195,8 @@ public SourceStatsManager(FunctionCollectorRegistry collectorRegistry, String[] .help("Exception from source.") .create()); - sysExceptionRateLimiter = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(5) - .rateTime(1) - .timeUnit(TimeUnit.MINUTES) - .build(); - sourceExceptionRateLimiter = RateLimiter.builder() - .scheduledExecutorService(scheduledExecutorService) - .permits(5) - .rateTime(1) - .timeUnit(TimeUnit.MINUTES) - .build(); + sysExceptionRateLimiter = RateLimiter.create(5.0d / 60.0d); + sourceExceptionRateLimiter = RateLimiter.create(5.0d / 60.0d); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java index 7f035b5562f24..e7816f06aacc8 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java @@ -41,7 +41,7 @@ public class LeaderService implements AutoCloseable, ConsumerEventListener { private ConsumerImpl consumer; private final WorkerConfig workerConfig; private final PulsarClient pulsarClient; - private boolean isLeader = false; + private volatile boolean isLeader = false; static final String COORDINATION_TOPIC_SUBSCRIPTION = "participants"; @@ -172,7 +172,7 @@ public synchronized void becameInactive(Consumer consumer, int partitionId) { } } - public synchronized boolean isLeader() { + public boolean isLeader() { return isLeader; } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 8a9ea22c53015..16cf778e07290 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -26,15 +26,14 @@ import java.io.IOException; import java.net.URI; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; import javax.ws.rs.core.Response; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.clients.StorageClientBuilder; -import org.apache.bookkeeper.clients.admin.StorageAdminClient; -import org.apache.bookkeeper.clients.config.StorageClientSettings; import org.apache.commons.lang3.StringUtils; import org.apache.distributedlog.DistributedLogConfiguration; import org.apache.distributedlog.api.namespace.Namespace; @@ -57,6 +56,7 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.SimpleTextOutputStream; +import org.apache.pulsar.functions.instance.state.StateStoreProvider; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; import org.apache.pulsar.functions.worker.rest.api.SinksImpl; @@ -97,7 +97,6 @@ public interface PulsarClientCreator { // dlog namespace for storing function jars in bookkeeper private Namespace dlogNamespace; // storage client for accessing state storage for functions - private StorageAdminClient stateStoreAdminClient; private MembershipManager membershipManager; private SchedulerManager schedulerManager; private volatile boolean isInitialized = false; @@ -121,6 +120,7 @@ public interface PulsarClientCreator { private Workers workers; private final PulsarClientCreator clientCreator; + private StateStoreProvider stateStoreProvider; public PulsarWorkerService() { this.clientCreator = new PulsarClientCreator() { @@ -418,14 +418,15 @@ public void start(AuthenticationService authenticationService, } } - // create the state storage client for accessing function state + // create the state storage provider for accessing function state if (workerConfig.getStateStorageServiceUrl() != null) { - StorageClientSettings clientSettings = StorageClientSettings.newBuilder() - .serviceUri(workerConfig.getStateStorageServiceUrl()) - .build(); - this.stateStoreAdminClient = StorageClientBuilder.newBuilder() - .withSettings(clientSettings) - .buildAdmin(); + this.stateStoreProvider = + (StateStoreProvider) Class.forName(workerConfig.getStateStorageProviderImplementation()) + .getConstructor().newInstance(); + Map stateStoreProviderConfig = new HashMap<>(); + stateStoreProviderConfig.put(StateStoreProvider.STATE_STORAGE_SERVICE_URL, + workerConfig.getStateStorageServiceUrl()); + this.stateStoreProvider.init(stateStoreProviderConfig); } final String functionWebServiceUrl = StringUtils.isNotBlank(workerConfig.getFunctionWebServiceUrl()) @@ -647,10 +648,6 @@ public void stop() { functionAdmin.close(); } - if (null != stateStoreAdminClient) { - stateStoreAdminClient.close(); - } - if (null != dlogNamespace) { dlogNamespace.close(); } @@ -658,6 +655,10 @@ public void stop() { if (statsUpdater != null) { statsUpdater.shutdownNow(); } + + if (null != stateStoreProvider) { + stateStoreProvider.close(); + } } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 585b54d846169..b175a7f275e71 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -20,17 +20,12 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.pulsar.functions.utils.FunctionCommon.createPkgTempFile; -import static org.apache.pulsar.functions.utils.FunctionCommon.getStateNamespace; import static org.apache.pulsar.functions.utils.FunctionCommon.getUniquePackageName; import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; import com.google.common.base.Utf8; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufUtil; -import io.netty.buffer.Unpooled; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -38,6 +33,7 @@ import java.io.InputStream; import java.net.URI; import java.net.URL; +import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -48,7 +44,6 @@ import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Response; @@ -56,14 +51,6 @@ import javax.ws.rs.core.StreamingOutput; import javax.ws.rs.core.UriBuilder; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.api.StorageClient; -import org.apache.bookkeeper.api.kv.Table; -import org.apache.bookkeeper.api.kv.result.KeyValue; -import org.apache.bookkeeper.clients.StorageClientBuilder; -import org.apache.bookkeeper.clients.admin.StorageAdminClient; -import org.apache.bookkeeper.clients.config.StorageClientSettings; -import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; -import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; @@ -88,6 +75,7 @@ import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.instance.InstanceUtils; +import org.apache.pulsar.functions.instance.state.DefaultStateStore; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.apache.pulsar.functions.proto.Function.FunctionMetaData; @@ -116,7 +104,6 @@ @Slf4j public abstract class ComponentImpl implements Component { - private final AtomicReference storageClient = new AtomicReference<>(); protected final Supplier workerServiceSupplier; protected final Function.FunctionDetails.ComponentType componentType; @@ -433,25 +420,6 @@ PackageLocationMetaData.Builder getFunctionPackageLocation(final FunctionMetaDat return packageLocationMetaDataBuilder; } - private void deleteStatestoreTableAsync(String namespace, String table) { - StorageAdminClient adminClient = worker().getStateStoreAdminClient(); - if (adminClient != null) { - adminClient.deleteStream(namespace, table).whenComplete((res, throwable) -> { - if ((throwable == null && res) - || ((throwable instanceof NamespaceNotFoundException - || throwable instanceof StreamNotFoundException))) { - log.info("{}/{} table deleted successfully", namespace, table); - } else { - if (throwable != null) { - log.error("{}/{} table deletion failed {} but moving on", namespace, table, throwable); - } else { - log.error("{}/{} table deletion failed but moving on", namespace, table); - } - } - }); - } - } - @Override public void deregisterFunction(final String tenant, final String namespace, @@ -508,7 +476,13 @@ public void deregisterFunction(final String tenant, functionMetaData.getTransformFunctionPackageLocation().getPackagePath()); } - deleteStatestoreTableAsync(getStateNamespace(tenant, namespace), componentName); + if (worker().getStateStoreProvider() != null) { + try { + worker().getStateStoreProvider().cleanUp(tenant, namespace, componentName); + } catch (Throwable e) { + log.error("failed to clean up the state store for {}/{}/{}", tenant, namespace, componentName); + } + } } private void deleteComponentFromStorage(String tenant, String namespace, String componentName, @@ -1162,7 +1136,7 @@ public FunctionState getFunctionState(final String tenant, throwRestExceptionIfUnauthorizedForNamespace(tenant, namespace, functionName, "get state for", authParams); - if (null == worker().getStateStoreAdminClient()) { + if (null == worker().getStateStoreProvider()) { throwStateStoreUnvailableResponse(); } @@ -1175,53 +1149,31 @@ public FunctionState getFunctionState(final String tenant, throw new RestException(Status.BAD_REQUEST, e.getMessage()); } - String tableNs = getStateNamespace(tenant, namespace); - String tableName = functionName; - - String stateStorageServiceUrl = worker().getWorkerConfig().getStateStorageServiceUrl(); + try { + DefaultStateStore store = worker().getStateStoreProvider().getStateStore(tenant, namespace, functionName); + ByteBuffer buf = store.get(key); + if (buf == null) { + throw new RestException(Status.NOT_FOUND, "key '" + key + "' doesn't exist."); + } - if (storageClient.get() == null) { - storageClient.compareAndSet(null, StorageClientBuilder.newBuilder() - .withSettings(StorageClientSettings.newBuilder() - .serviceUri(stateStorageServiceUrl) - .clientName("functions-admin") - .build()) - .withNamespace(tableNs) - .build()); - } + // try to parse the state as a long + // but even if it can be parsed as a long, this number may not be the actual state, + // so we will always return a `stringValue` or `bytesValue` with the number value + Long number = null; + if (buf.remaining() == Long.BYTES) { + number = buf.getLong(); + } - FunctionState value; - try (Table table = result(storageClient.get().openTable(tableName))) { - try (KeyValue kv = result(table.getKv(Unpooled.wrappedBuffer(key.getBytes(UTF_8))))) { - if (null == kv) { - throw new RestException(Status.NOT_FOUND, "key '" + key + "' doesn't exist."); - } else { - if (kv.isNumber()) { - value = new FunctionState(key, null, null, kv.numberValue(), kv.version()); - } else { - byte[] bytes = ByteBufUtil.getBytes(kv.value()); - if (Utf8.isWellFormed(bytes)) { - value = new FunctionState(key, new String(bytes, UTF_8), - null, null, kv.version()); - } else { - value = new FunctionState( - key, null, bytes, null, kv.version()); - } - } - } + if (Utf8.isWellFormed(buf.array())) { + return new FunctionState(key, new String(buf.array(), UTF_8), null, number, null); + } else { + return new FunctionState(key, null, buf.array(), number, null); } - } catch (RestException e) { - throw e; - } catch (org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException | StreamNotFoundException e) { - log.debug("State not found while processing getFunctionState request @ /{}/{}/{}/{}", - tenant, namespace, functionName, key, e); - throw new RestException(Status.NOT_FOUND, e.getMessage()); - } catch (Exception e) { + } catch (Throwable e) { log.error("Error while getFunctionState request @ /{}/{}/{}/{}", tenant, namespace, functionName, key, e); throw new RestException(Status.INTERNAL_SERVER_ERROR, e.getMessage()); } - return value; } @Override @@ -1236,7 +1188,7 @@ public void putFunctionState(final String tenant, throwUnavailableException(); } - if (null == worker().getStateStoreAdminClient()) { + if (null == worker().getStateStoreProvider()) { throwStateStoreUnvailableResponse(); } @@ -1248,9 +1200,6 @@ public void putFunctionState(final String tenant, functionName); throw new RestException(Status.BAD_REQUEST, "Path key doesn't match key in json"); } - if (state.getStringValue() == null && state.getByteValue() == null) { - throw new RestException(Status.BAD_REQUEST, "Setting Counter values not supported in put state"); - } // validate parameters try { @@ -1261,35 +1210,21 @@ public void putFunctionState(final String tenant, throw new RestException(Status.BAD_REQUEST, e.getMessage()); } - String tableNs = getStateNamespace(tenant, namespace); - String tableName = functionName; - - String stateStorageServiceUrl = worker().getWorkerConfig().getStateStorageServiceUrl(); - - if (storageClient.get() == null) { - storageClient.compareAndSet(null, StorageClientBuilder.newBuilder() - .withSettings(StorageClientSettings.newBuilder() - .serviceUri(stateStorageServiceUrl) - .clientName("functions-admin") - .build()) - .withNamespace(tableNs) - .build()); - } - - ByteBuf value; - if (!isEmpty(state.getStringValue())) { - value = Unpooled.wrappedBuffer(state.getStringValue().getBytes()); - } else { - value = Unpooled.wrappedBuffer(state.getByteValue()); - } - try (Table table = result(storageClient.get().openTable(tableName))) { - result(table.put(Unpooled.wrappedBuffer(key.getBytes(UTF_8)), value)); - } catch (org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException - | org.apache.bookkeeper.clients.exceptions.StreamNotFoundException e) { - log.debug("State not found while processing putFunctionState request @ /{}/{}/{}/{}", - tenant, namespace, functionName, key, e); - throw new RestException(Status.NOT_FOUND, e.getMessage()); - } catch (Exception e) { + try { + DefaultStateStore store = worker().getStateStoreProvider().getStateStore(tenant, namespace, functionName); + ByteBuffer data; + if (StringUtils.isNotEmpty(state.getStringValue())) { + data = ByteBuffer.wrap(state.getStringValue().getBytes(UTF_8)); + } else if (state.getByteValue() != null) { + data = ByteBuffer.wrap(state.getByteValue()); + } else if (state.getNumberValue() != null) { + data = ByteBuffer.allocate(Long.BYTES); + data.putLong(state.getNumberValue()); + } else { + throw new IllegalArgumentException("Invalid state value"); + } + store.put(key, data); + } catch (Throwable e) { log.error("Error while putFunctionState request @ /{}/{}/{}/{}", tenant, namespace, functionName, key, e); throw new RestException(Status.INTERNAL_SERVER_ERROR, e.getMessage()); diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 7525f7a6368ce..8abf89a98aab2 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -29,9 +29,10 @@ - 2.7.3 + 2.9.3 4.1.11 1.37.0 + 4.1.100.Final pulsar-io-alluxio @@ -56,12 +57,6 @@ org.alluxio alluxio-core-client-fs ${alluxio.version} - - - grpc-netty - io.grpc - - @@ -74,10 +69,6 @@ org.glassfish javax.el - - grpc-netty - io.grpc - @@ -90,22 +81,32 @@ com.google.guava guava - - - - io.grpc - grpc-netty - ${grpc.version} - - - - io.dropwizard.metrics - metrics-jvm - ${metrics.version} - - + + + + io.netty + netty-bom + ${netty.version} + pom + import + + + io.grpc + grpc-bom + ${grpc.version} + pom + import + + + io.dropwizard.metrics + metrics-jvm + ${metrics.version} + + + + diff --git a/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java b/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java index 413f05e0e17c5..3b72dc9666b78 100644 --- a/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java +++ b/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java @@ -22,12 +22,13 @@ import alluxio.client.WriteType; import alluxio.client.file.FileOutStream; import alluxio.client.file.FileSystem; +import alluxio.conf.Configuration; import alluxio.conf.InstancedConfiguration; import alluxio.conf.PropertyKey; import alluxio.exception.AlluxioException; import alluxio.grpc.CreateFilePOptions; import alluxio.grpc.WritePType; -import alluxio.util.FileSystemOptions; +import alluxio.util.FileSystemOptionsUtils; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -78,7 +79,7 @@ public class AlluxioSink implements Sink { private AlluxioSinkConfig alluxioSinkConfig; private AlluxioState alluxioState; - private InstancedConfiguration configuration = InstancedConfiguration.defaults(); + private InstancedConfiguration configuration = Configuration.modifiableGlobal(); private ObjectMapper objectMapper = new ObjectMapper(); @@ -205,7 +206,7 @@ private void writeToAlluxio(Record record) throws AlluxioExceptio private void createTmpFile() throws AlluxioException, IOException { CreateFilePOptions.Builder optionsBuilder = - FileSystemOptions.createFileDefaults(configuration).toBuilder(); + FileSystemOptionsUtils.createFileDefaults(configuration).toBuilder(); UUID id = UUID.randomUUID(); String fileExtension = alluxioSinkConfig.getFileExtension(); tmpFilePath = tmpFileDirPath + "/" + id.toString() + "_tmp" + fileExtension; diff --git a/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java b/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java index 9325a2255ab0a..bf40581aae155 100644 --- a/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java +++ b/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java @@ -22,8 +22,8 @@ import alluxio.client.WriteType; import alluxio.client.file.FileSystem; import alluxio.client.file.URIStatus; +import alluxio.conf.Configuration; import alluxio.conf.PropertyKey; -import alluxio.conf.ServerConfiguration; import alluxio.master.LocalAlluxioCluster; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.FilenameUtils; @@ -237,8 +237,8 @@ public Object getNativeObject() { private LocalAlluxioCluster setupSingleMasterCluster() throws Exception { // Setup and start the local alluxio cluster LocalAlluxioCluster cluster = new LocalAlluxioCluster(); - cluster.initConfiguration(getTestName(getClass().getSimpleName(), LocalAlluxioCluster.DEFAULT_TEST_NAME)); - ServerConfiguration.set(PropertyKey.USER_FILE_WRITE_TYPE_DEFAULT, WriteType.MUST_CACHE); + cluster.initConfiguration(getTestName(getClass().getSimpleName(), "test")); + Configuration.set(PropertyKey.USER_FILE_WRITE_TYPE_DEFAULT, WriteType.MUST_CACHE); cluster.start(); return cluster; } diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index 03ea9729e885c..696f5f8e970a1 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -37,6 +37,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java index 06c8788d5aea1..7d0cd0305a49e 100644 --- a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java +++ b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java @@ -57,7 +57,7 @@ public abstract class CanalAbstractSource extends PushSource { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - canalSourceConfig = CanalSourceConfig.load(config); + canalSourceConfig = CanalSourceConfig.load(config, sourceContext); if (canalSourceConfig.getCluster()) { connector = CanalConnectors.newClusterConnector(canalSourceConfig.getZkServers(), canalSourceConfig.getDestination(), canalSourceConfig.getUsername(), diff --git a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java index a0408e60e5f76..5a754988ffdc1 100644 --- a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java +++ b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java @@ -26,6 +26,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @@ -86,8 +88,7 @@ public static CanalSourceConfig load(String yamlFile) throws IOException { } - public static CanalSourceConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), CanalSourceConfig.class); + public static CanalSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, CanalSourceConfig.class, sourceContext); } } diff --git a/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java b/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java index d15986a897caa..69d981bf68728 100644 --- a/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java +++ b/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java @@ -77,13 +77,14 @@ private static T loadWithSecrets(Map map, Class clazz, } } configs.computeIfAbsent(field.getName(), key -> { - if (fieldDoc.required()) { - throw new IllegalArgumentException(field.getName() + " cannot be null"); - } + // Use default value if it is not null before checking required String value = fieldDoc.defaultValue(); if (value != null && !value.isEmpty()) { return value; } + if (fieldDoc.required()) { + throw new IllegalArgumentException(field.getName() + " cannot be null"); + } return null; }); } diff --git a/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java b/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java index fd291a8a3c9b3..cd31a1a4f066a 100644 --- a/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java +++ b/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java @@ -54,6 +54,14 @@ static class TestDefaultConfig { ) protected String testRequired; + @FieldDoc( + required = true, + defaultValue = "defaultRequired", + sensitive = true, + help = "testRequired" + ) + protected String testDefaultRequired; + @FieldDoc( required = false, defaultValue = "defaultStr", @@ -304,6 +312,9 @@ public void testDefaultValue() { configMap.put("testRequired", "test"); TestDefaultConfig testDefaultConfig = IOConfigUtils.loadWithSecrets(configMap, TestDefaultConfig.class, new TestSinkContext()); + // if there is default value for a required field and no value provided when load config, + // it should not throw exception but use the default value. + Assert.assertEquals(testDefaultConfig.getTestDefaultRequired(), "defaultRequired"); Assert.assertEquals(testDefaultConfig.getDefaultStr(), "defaultStr"); Assert.assertEquals(testDefaultConfig.isDefaultBool(), true); Assert.assertEquals(testDefaultConfig.getDefaultInt(), 100); diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index bf50a4289994d..c88c7fe7a6cc9 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -32,6 +32,12 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java index d67c4e21154ee..2193cf39c17a5 100644 --- a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java +++ b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java @@ -65,7 +65,7 @@ public void close() throws Exception { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - this.dynamodbSourceConfig = DynamoDBSourceConfig.load(config); + this.dynamodbSourceConfig = DynamoDBSourceConfig.load(config, sourceContext); checkArgument(isNotBlank(dynamodbSourceConfig.getAwsDynamodbStreamArn()), "empty dynamo-stream arn"); // Even if the endpoint is set, it seems to require a region to go with it diff --git a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java index b734dd5741155..0547ff8f863e0 100644 --- a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java +++ b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java @@ -35,6 +35,8 @@ import java.util.Map; import lombok.Data; import org.apache.pulsar.io.aws.AwsCredentialProviderPlugin; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; import software.amazon.awssdk.regions.Region; @@ -77,6 +79,7 @@ public class DynamoDBSourceConfig implements Serializable { @FieldDoc( required = false, defaultValue = "", + sensitive = true, help = "json-parameters to initialize `AwsCredentialsProviderPlugin`") private String awsCredentialPluginParam = ""; @@ -170,9 +173,8 @@ public static DynamoDBSourceConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), DynamoDBSourceConfig.class); } - public static DynamoDBSourceConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), DynamoDBSourceConfig.class); + public static DynamoDBSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, DynamoDBSourceConfig.class, sourceContext); } protected Region regionAsV2Region() { diff --git a/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java b/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java index f84cb785896e6..bdccaa2e5846e 100644 --- a/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java +++ b/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java @@ -31,6 +31,8 @@ import java.util.Map; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.apache.pulsar.io.core.SourceContext; +import org.mockito.Mockito; import org.testng.annotations.Test; @@ -90,7 +92,8 @@ public final void loadFromMapTest() throws IOException { map.put("initialPositionInStream", InitialPositionInStream.TRIM_HORIZON); map.put("startAtTime", DAY); - DynamoDBSourceConfig config = DynamoDBSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + DynamoDBSourceConfig config = DynamoDBSourceConfig.load(map, sourceContext); assertNotNull(config); assertEquals(config.getAwsEndpoint(), "https://some.endpoint.aws"); @@ -111,7 +114,46 @@ public final void loadFromMapTest() throws IOException { ZonedDateTime expected = ZonedDateTime.ofInstant(DAY.toInstant(), ZoneOffset.UTC); assertEquals(actual, expected); } - + + @Test + public final void loadFromMapCredentialFromSecretTest() throws IOException { + Map map = new HashMap (); + map.put("awsEndpoint", "https://some.endpoint.aws"); + map.put("awsRegion", "us-east-1"); + map.put("awsDynamodbStreamArn", "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291"); + map.put("checkpointInterval", "30000"); + map.put("backoffTime", "4000"); + map.put("numRetries", "3"); + map.put("receiveQueueSize", 2000); + map.put("applicationName", "My test application"); + map.put("initialPositionInStream", InitialPositionInStream.TRIM_HORIZON); + map.put("startAtTime", DAY); + + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("awsCredentialPluginParam")) + .thenReturn("{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}"); + DynamoDBSourceConfig config = DynamoDBSourceConfig.load(map, sourceContext); + + assertNotNull(config); + assertEquals(config.getAwsEndpoint(), "https://some.endpoint.aws"); + assertEquals(config.getAwsRegion(), "us-east-1"); + assertEquals(config.getAwsDynamodbStreamArn(), "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291"); + assertEquals(config.getAwsCredentialPluginParam(), + "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}"); + assertEquals(config.getApplicationName(), "My test application"); + assertEquals(config.getCheckpointInterval(), 30000); + assertEquals(config.getBackoffTime(), 4000); + assertEquals(config.getNumRetries(), 3); + assertEquals(config.getReceiveQueueSize(), 2000); + assertEquals(config.getInitialPositionInStream(), InitialPositionInStream.TRIM_HORIZON); + + Calendar cal = Calendar.getInstance(); + cal.setTime(config.getStartAtTime()); + ZonedDateTime actual = ZonedDateTime.ofInstant(cal.toInstant(), ZoneOffset.UTC); + ZonedDateTime expected = ZonedDateTime.ofInstant(DAY.toInstant(), ZoneOffset.UTC); + assertEquals(actual, expected); + } + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "empty aws-credential param") public final void missingCredentialsTest() throws Exception { @@ -121,7 +163,8 @@ public final void missingCredentialsTest() throws Exception { map.put("awsDynamodbStreamArn", "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291"); DynamoDBSource source = new DynamoDBSource(); - source.open(map, null); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + source.open(map, sourceContext); } @Test(expectedExceptions = IllegalArgumentException.class, @@ -136,7 +179,8 @@ public final void missingStartTimeTest() throws Exception { map.put("initialPositionInStream", InitialPositionInStream.AT_TIMESTAMP); DynamoDBSource source = new DynamoDBSource(); - source.open(map, null); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + source.open(map, sourceContext); } private File getFile(String name) { diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchExtractTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchExtractTests.java index cbc3de908c68f..f5a2e36aef44d 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchExtractTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchExtractTests.java @@ -101,6 +101,7 @@ public GenericObject getValue() { Pair pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getLeft(), "1"); assertEquals(pair.getRight(), "{\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink.close(); // two fields PK ElasticSearchSink elasticSearchSink2 = new ElasticSearchSink(); @@ -113,6 +114,7 @@ public GenericObject getValue() { Pair pair2 = elasticSearchSink2.extractIdAndDocument(genericObjectRecord); assertEquals(pair2.getLeft(), "[\"1\",1]"); assertEquals(pair2.getRight(), "{\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink2.close(); // default config with null PK => indexed with auto generated _id ElasticSearchSink elasticSearchSink3 = new ElasticSearchSink(); @@ -122,6 +124,7 @@ public GenericObject getValue() { Pair pair3 = elasticSearchSink3.extractIdAndDocument(genericObjectRecord); assertNull(pair3.getLeft()); assertEquals(pair3.getRight(), "{\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink3.close(); // default config with null PK + null value ElasticSearchSink elasticSearchSink4 = new ElasticSearchSink(); @@ -146,6 +149,7 @@ public GenericObject getValue() { }); assertNull(pair4.getLeft()); assertNull(pair4.getRight()); + elasticSearchSink4.close(); } @Test(dataProvider = "schemaType") @@ -225,6 +229,7 @@ public GenericObject getValue() { Pair pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getLeft(), "[\"1\",1]"); assertEquals(pair.getRight(), "{\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink.close(); elasticSearchSink = new ElasticSearchSink(); elasticSearchSink.open(ImmutableMap.of( @@ -236,6 +241,7 @@ public GenericObject getValue() { pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getLeft(), "[\"1\",1]"); assertEquals(pair.getRight(), "{\"a\":\"1\",\"b\":1,\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink.close(); elasticSearchSink = new ElasticSearchSink(); elasticSearchSink.open(ImmutableMap.of( @@ -246,6 +252,7 @@ public GenericObject getValue() { pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertNull(pair.getLeft()); assertEquals(pair.getRight(), "{\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink.close(); elasticSearchSink = new ElasticSearchSink(); elasticSearchSink.open(ImmutableMap.of("elasticSearchUrl", "http://localhost:9200", @@ -255,6 +262,7 @@ public GenericObject getValue() { pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertNull(pair.getLeft()); assertEquals(pair.getRight(), "{\"a\":\"1\",\"b\":1,\"c\":\"1\",\"d\":1,\"e\":{\"a\":\"a\",\"b\":true,\"d\":1.0,\"f\":1.0,\"i\":1,\"l\":10}}"); + elasticSearchSink.close(); // test null value elasticSearchSink = new ElasticSearchSink(); @@ -291,6 +299,7 @@ public Object getNativeObject() { }); assertEquals(pair.getLeft(), "[\"1\",1]"); assertNull(pair.getRight()); + elasticSearchSink.close(); } @Test(dataProvider = "schemaType") @@ -326,6 +335,7 @@ public void testSortKeysSingle(SchemaType schemaType) throws Exception { "keyIgnore", "false"), null); Pair pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getKey(), "{\"b_inside_inner\":\"0b_value_from_inner\",\"a_inside_inner\":\"a_value_from_inner\"}"); + elasticSearchSink.close(); elasticSearchSink = new ElasticSearchSink(); elasticSearchSink.open(ImmutableMap.of( @@ -336,6 +346,7 @@ public void testSortKeysSingle(SchemaType schemaType) throws Exception { "keyIgnore", "false"), null); pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getKey(), "{\"a_inside_inner\":\"a_value_from_inner\",\"b_inside_inner\":\"0b_value_from_inner\"}"); + elasticSearchSink.close(); } @@ -378,6 +389,7 @@ public void testSortKeysMulti(SchemaType schemaType) throws Exception { "keyIgnore", "false"), null); Pair pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getKey(), "[\"a_key\",\"0b_key\",\"c_key\",{\"b_inside_inner\":\"0b_value_from_inner\",\"a_inside_inner\":\"a_value_from_inner\"}]"); + elasticSearchSink.close(); elasticSearchSink = new ElasticSearchSink(); elasticSearchSink.open(ImmutableMap.of( @@ -388,6 +400,7 @@ public void testSortKeysMulti(SchemaType schemaType) throws Exception { "keyIgnore", "false"), null); pair = elasticSearchSink.extractIdAndDocument(genericObjectRecord); assertEquals(pair.getKey(), "[\"a_key\",\"0b_key\",\"c_key\",{\"a_inside_inner\":\"a_value_from_inner\",\"b_inside_inner\":\"0b_value_from_inner\"}]"); + elasticSearchSink.close(); } private Record getKeyValueGenericObject(SchemaType schemaType, GenericSchema keySchema, GenericRecord keyGenericRecord) { diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index ac2be2dfd2459..d800a87960169 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -55,14 +55,18 @@ hadoop-client ${hadoop2.version} - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.avro + avro + diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index e0f4ca58e878b..e52f7a68f602b 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -67,6 +67,10 @@ org.slf4j slf4j-log4j12 + + org.apache.avro + avro + diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index 994524c197001..5f7e550989ef3 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -32,6 +32,11 @@ Pulsar IO :: InfluxDB + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java index 5b51461fc7b8e..0d431f84c52f2 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java @@ -46,12 +46,12 @@ public class InfluxDBGenericRecordSink implements Sink { @Override public void open(Map map, SinkContext sinkContext) throws Exception { try { - val configV2 = InfluxDBSinkConfig.load(map); + val configV2 = InfluxDBSinkConfig.load(map, sinkContext); configV2.validate(); sink = new InfluxDBSink(); } catch (Exception e) { try { - val configV1 = org.apache.pulsar.io.influxdb.v1.InfluxDBSinkConfig.load(map); + val configV1 = org.apache.pulsar.io.influxdb.v1.InfluxDBSinkConfig.load(map, sinkContext); configV1.validate(); sink = new org.apache.pulsar.io.influxdb.v1.InfluxDBGenericRecordSink(); } catch (Exception e1) { diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java index 06856bad80edc..217c5304b24f7 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java @@ -43,7 +43,7 @@ public abstract class InfluxDBAbstractSink extends BatchSink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config); + InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config, sinkContext); influxDBSinkConfig.validate(); super.init(influxDBSinkConfig.getBatchTimeMs(), influxDBSinkConfig.getBatchSize()); diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java index 9b7d8e1ce905d..4ae2cf1e4a3a1 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java @@ -27,6 +27,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -94,7 +96,7 @@ public class InfluxDBSinkConfig implements Serializable { @FieldDoc( required = false, - defaultValue = "1000L", + defaultValue = "1000", help = "The InfluxDB operation time in milliseconds") private long batchTimeMs = 1000L; @@ -110,14 +112,11 @@ public static InfluxDBSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), InfluxDBSinkConfig.class); } - public static InfluxDBSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), InfluxDBSinkConfig.class); + public static InfluxDBSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, InfluxDBSinkConfig.class, sinkContext); } public void validate() { - Preconditions.checkNotNull(influxdbUrl, "influxdbUrl property not set."); - Preconditions.checkNotNull(database, "database property not set."); Preconditions.checkArgument(batchSize > 0, "batchSize must be a positive integer."); Preconditions.checkArgument(batchTimeMs > 0, "batchTimeMs must be a positive long."); } diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java index 08f1ab2339992..0aa43570596af 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java @@ -49,7 +49,7 @@ public class InfluxDBSink extends BatchSink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config); + InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config, sinkContext); influxDBSinkConfig.validate(); super.init(influxDBSinkConfig.getBatchTimeMs(), influxDBSinkConfig.getBatchSize()); diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java index 899b00c002155..ea87ee66b90a3 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java @@ -27,6 +27,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -87,7 +89,7 @@ public class InfluxDBSinkConfig implements Serializable { @FieldDoc( required = false, - defaultValue = "1000L", + defaultValue = "1000", help = "The InfluxDB operation time in milliseconds") private long batchTimeMs = 1000; @@ -103,17 +105,11 @@ public static InfluxDBSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), InfluxDBSinkConfig.class); } - public static InfluxDBSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), InfluxDBSinkConfig.class); + public static InfluxDBSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, InfluxDBSinkConfig.class, sinkContext); } public void validate() { - Preconditions.checkNotNull(influxdbUrl, "influxdbUrl property not set."); - Preconditions.checkNotNull(token, "token property not set."); - Preconditions.checkNotNull(organization, "organization property not set."); - Preconditions.checkNotNull(bucket, "bucket property not set."); - Preconditions.checkArgument(batchSize > 0, "batchSize must be a positive integer."); Preconditions.checkArgument(batchTimeMs > 0, "batchTimeMs must be a positive long."); } diff --git a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java index 4493dcfb24854..10b1bfb624f49 100644 --- a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java +++ b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.influxdb.v1; +import org.apache.pulsar.io.core.SinkContext; import org.influxdb.InfluxDB; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -60,8 +62,11 @@ public final void loadFromMapTest() throws IOException { map.put("gzipEnable", "false"); map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); + map.put("username", "admin"); + map.put("password", "admin"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals("http://localhost:8086", config.getInfluxdbUrl()); assertEquals("test_db", config.getDatabase()); @@ -71,6 +76,39 @@ public final void loadFromMapTest() throws IOException { assertEquals(Boolean.parseBoolean("false"), config.isGzipEnable()); assertEquals(Long.parseLong("1000"), config.getBatchTimeMs()); assertEquals(Integer.parseInt("100"), config.getBatchSize()); + assertEquals("admin", config.getUsername()); + assertEquals("admin", config.getPassword()); + } + + @Test + public final void loadFromMapCredentialFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("influxdbUrl", "http://localhost:8086"); + map.put("database", "test_db"); + map.put("consistencyLevel", "ONE"); + map.put("logLevel", "NONE"); + map.put("retentionPolicy", "autogen"); + map.put("gzipEnable", "false"); + map.put("batchTimeMs", "1000"); + map.put("batchSize", "100"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("username")) + .thenReturn("admin"); + Mockito.when(sinkContext.getSecret("password")) + .thenReturn("admin"); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals("http://localhost:8086", config.getInfluxdbUrl()); + assertEquals("test_db", config.getDatabase()); + assertEquals("ONE", config.getConsistencyLevel()); + assertEquals("NONE", config.getLogLevel()); + assertEquals("autogen", config.getRetentionPolicy()); + assertEquals(Boolean.parseBoolean("false"), config.isGzipEnable()); + assertEquals(Long.parseLong("1000"), config.getBatchTimeMs()); + assertEquals(Integer.parseInt("100"), config.getBatchSize()); + assertEquals("admin", config.getUsername()); + assertEquals("admin", config.getPassword()); } @Test @@ -85,12 +123,13 @@ public final void validValidateTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "influxdbUrl property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "influxdbUrl cannot be null") public final void missingInfluxdbUrlValidateTest() throws IOException { Map map = new HashMap<>(); map.put("database", "test_db"); @@ -101,7 +140,8 @@ public final void missingInfluxdbUrlValidateTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } @@ -118,7 +158,8 @@ public final void invalidBatchSizeTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "-100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } @@ -135,7 +176,8 @@ public final void invalidConsistencyLevelTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); InfluxDB.ConsistencyLevel.valueOf(config.getConsistencyLevel().toUpperCase()); diff --git a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java index df1f7fd29a637..d6cee1e308d2b 100644 --- a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java +++ b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java @@ -24,6 +24,8 @@ import java.io.File; import java.util.HashMap; import java.util.Map; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testng.annotations.Test; public class InfluxDBSinkConfigTest { @@ -58,18 +60,34 @@ private Map buildValidConfigMap() { public final void testLoadFromMap() throws Exception { Map map = buildValidConfigMap(); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); assertNotNull(config); config.validate(); verifyValues(config); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "influxdbUrl property not set.") + @Test + public final void testLoadFromMapCredentialFromSecret() throws Exception { + Map map = buildValidConfigMap(); + map.remove("token"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("token")) + .thenReturn("xxxx"); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); + assertNotNull(config); + config.validate(); + verifyValues(config); + } + + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "influxdbUrl cannot be null") public void testRequiredConfigMissing() throws Exception { Map map = buildValidConfigMap(); map.remove("influxdbUrl"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } @@ -78,7 +96,8 @@ public void testRequiredConfigMissing() throws Exception { public void testBatchConfig() throws Exception { Map map = buildValidConfigMap(); map.put("batchSize", -1); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 18085bafff205..ae4e4f77d5386 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -32,6 +32,12 @@ Pulsar IO :: Jdbc :: Core + + ${project.groupId} + pulsar-io-common + ${project.version} + + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java index 4586fcebcf167..ca33b3cfdaba9 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java @@ -76,7 +76,7 @@ public abstract class JdbcAbstractSink implements Sink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - jdbcSinkConfig = JdbcSinkConfig.load(config); + jdbcSinkConfig = JdbcSinkConfig.load(config, sinkContext); jdbcSinkConfig.validate(); jdbcUrl = jdbcSinkConfig.getJdbcUrl(); diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java index f798d94f7c35e..854d68381312c 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java @@ -26,6 +26,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -145,9 +147,8 @@ public static JdbcSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), JdbcSinkConfig.class); } - public static JdbcSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), JdbcSinkConfig.class); + public static JdbcSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, JdbcSinkConfig.class, sinkContext); } public void validate() { diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 7f591295d76f5..8ad503a72ce50 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -46,6 +46,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java index 5ceea4dec8dca..2bedba928b756 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Map; -import java.util.Objects; import java.util.Properties; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -79,10 +78,7 @@ protected Properties beforeCreateProducer(Properties props) { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - kafkaSinkConfig = KafkaSinkConfig.load(config); - Objects.requireNonNull(kafkaSinkConfig.getTopic(), "Kafka topic is not set"); - Objects.requireNonNull(kafkaSinkConfig.getBootstrapServers(), "Kafka bootstrapServers is not set"); - Objects.requireNonNull(kafkaSinkConfig.getAcks(), "Kafka acks mode is not set"); + kafkaSinkConfig = KafkaSinkConfig.load(config, sinkContext); if (kafkaSinkConfig.getBatchSize() <= 0) { throw new IllegalArgumentException("Invalid Kafka Producer batchSize : " + kafkaSinkConfig.getBatchSize()); diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java index f8539518851aa..782f9d5d57dbb 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java @@ -66,7 +66,7 @@ public abstract class KafkaAbstractSource extends PushSource { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - kafkaSourceConfig = KafkaSourceConfig.load(config); + kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); Objects.requireNonNull(kafkaSourceConfig.getTopic(), "Kafka topic is not set"); Objects.requireNonNull(kafkaSourceConfig.getBootstrapServers(), "Kafka bootstrapServers is not set"); Objects.requireNonNull(kafkaSourceConfig.getGroupId(), "Kafka consumer group id is not set"); diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java index 1b5878ff06c19..b63e3756693bb 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java @@ -26,6 +26,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -85,12 +87,12 @@ public class KafkaSinkConfig implements Serializable { + " before considering a request complete. This controls the durability of records that are sent.") private String acks; @FieldDoc( - defaultValue = "16384L", + defaultValue = "16384", help = "The batch size that Kafka producer will attempt to batch records together" + " before sending them to brokers.") private long batchSize = 16384L; @FieldDoc( - defaultValue = "1048576L", + defaultValue = "1048576", help = "The maximum size of a Kafka request in bytes.") private long maxRequestSize = 1048576L; @@ -123,8 +125,7 @@ public static KafkaSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), KafkaSinkConfig.class); } - public static KafkaSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), KafkaSinkConfig.class); + public static KafkaSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, KafkaSinkConfig.class, sinkContext); } } diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java index 7065458649c83..bc278ce22c64c 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java @@ -27,6 +27,7 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -152,8 +153,14 @@ public static KafkaSourceConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), KafkaSourceConfig.class); } - public static KafkaSourceConfig load(Map map) throws IOException { + public static KafkaSourceConfig load(Map map, SourceContext sourceContext) throws IOException { ObjectMapper mapper = new ObjectMapper(); + // since the KafkaSourceConfig requires the ACCEPT_EMPTY_STRING_AS_NULL_OBJECT feature + // We manually set the sensitive fields here instead of calling `IOConfigUtils.loadWithSecrets` + String sslTruststorePassword = sourceContext.getSecret("sslTruststorePassword"); + if (sslTruststorePassword != null) { + map.put("sslTruststorePassword", sslTruststorePassword); + } mapper.enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT); return mapper.readValue(mapper.writeValueAsString(map), KafkaSourceConfig.class); } diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java index 9537b6576b44e..0f6920690d6cf 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java @@ -193,12 +193,12 @@ public void fatal(Throwable t) { sink.close(); } }; - expectThrows(NullPointerException.class, "Kafka topic is not set", openAndClose); - config.put("topic", "topic_2"); - expectThrows(NullPointerException.class, "Kafka bootstrapServers is not set", openAndClose); + expectThrows(IllegalArgumentException.class, "bootstrapServers cannot be null", openAndClose); config.put("bootstrapServers", "localhost:6667"); - expectThrows(NullPointerException.class, "Kafka acks mode is not set", openAndClose); + expectThrows(IllegalArgumentException.class, "acks cannot be null", openAndClose); config.put("acks", "1"); + expectThrows(IllegalArgumentException.class, "topic cannot be null", openAndClose); + config.put("topic", "topic_2"); config.put("batchSize", "-1"); expectThrows(IllegalArgumentException.class, "Invalid Kafka Producer batchSize : -1", openAndClose); config.put("batchSize", "16384"); diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 9e0fef87a2592..7675de0636e8a 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -108,19 +108,39 @@ public void testInvalidConfigWillThrownException() throws Exception { public void loadConsumerConfigPropertiesFromMapTest() throws Exception { Map config = new HashMap<>(); config.put("consumerConfigProperties", ""); - KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.load(config); + config.put("bootstrapServers", "localhost:8080"); + config.put("groupId", "test-group"); + config.put("topic", "test-topic"); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); assertNotNull(kafkaSourceConfig); assertNull(kafkaSourceConfig.getConsumerConfigProperties()); config.put("consumerConfigProperties", null); - kafkaSourceConfig = KafkaSourceConfig.load(config); + kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); assertNull(kafkaSourceConfig.getConsumerConfigProperties()); config.put("consumerConfigProperties", ImmutableMap.of("foo", "bar")); - kafkaSourceConfig = KafkaSourceConfig.load(config); + kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); assertEquals(kafkaSourceConfig.getConsumerConfigProperties(), ImmutableMap.of("foo", "bar")); } + @Test + public void loadSensitiveFieldsFromSecretTest() throws Exception { + Map config = new HashMap<>(); + config.put("consumerConfigProperties", ""); + config.put("bootstrapServers", "localhost:8080"); + config.put("groupId", "test-group"); + config.put("topic", "test-topic"); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("sslTruststorePassword")) + .thenReturn("xxxx"); + KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); + assertNotNull(kafkaSourceConfig); + assertNull(kafkaSourceConfig.getConsumerConfigProperties()); + assertEquals("xxxx", kafkaSourceConfig.getSslTruststorePassword()); + } + @Test public final void loadFromYamlFileTest() throws IOException { File yamlFile = getFile("kafkaSourceConfig.yaml"); diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index 1ca1cdf77ad65..7ad363f3b381a 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -37,6 +37,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.parent.groupId} pulsar-io-core diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java index 35c327ed82b99..74f077da62036 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java @@ -24,7 +24,6 @@ import java.io.Serializable; import lombok.Data; import lombok.experimental.Accessors; -import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -42,6 +41,7 @@ public abstract class MongoAbstractConfig implements Serializable { @FieldDoc( required = true, + sensitive = true, // it may contain password defaultValue = "", help = "The URI of MongoDB that the connector connects to " + "(see: https://docs.mongodb.com/manual/reference/connection-string/)" @@ -95,7 +95,6 @@ public MongoAbstractConfig( } public void validate() { - checkArgument(!StringUtils.isEmpty(getMongoUri()), "Required MongoDB URI is not set."); checkArgument(getBatchSize() > 0, "batchSize must be a positive integer."); checkArgument(getBatchTimeMs() > 0, "batchTimeMs must be a positive long."); } diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java index 2206d232eaf97..61d5aeb697e01 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java @@ -86,7 +86,7 @@ public MongoSink(Supplier clientProvider) { public void open(Map config, SinkContext sinkContext) throws Exception { log.info("Open MongoDB Sink"); - mongoSinkConfig = MongoSinkConfig.load(config); + mongoSinkConfig = MongoSinkConfig.load(config, sinkContext); mongoSinkConfig.validate(); if (clientProvider != null) { diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java index 285f3c97bef1a..9431fe4910800 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java @@ -30,6 +30,8 @@ import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; /** * Configuration class for the MongoDB Sink Connectors. @@ -59,11 +61,8 @@ public static MongoSinkConfig load(String yamlFile) throws IOException { return cfg; } - public static MongoSinkConfig load(Map map) throws IOException { - final ObjectMapper mapper = new ObjectMapper(); - final MongoSinkConfig cfg = mapper.readValue(mapper.writeValueAsString(map), MongoSinkConfig.class); - - return cfg; + public static MongoSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, MongoSinkConfig.class, sinkContext); } @Override diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java index 6ee95fc4cd4b5..68a31b461a51c 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java @@ -79,7 +79,7 @@ public MongoSource(Supplier clientProvider) { public void open(Map config, SourceContext sourceContext) throws Exception { log.info("Open MongoDB Source"); - mongoSourceConfig = MongoSourceConfig.load(config); + mongoSourceConfig = MongoSourceConfig.load(config, sourceContext); mongoSourceConfig.validate(); if (clientProvider != null) { diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java index cf887a93bf3c3..1c0c7f4b3657a 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java @@ -29,6 +29,8 @@ import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -75,12 +77,8 @@ public static MongoSourceConfig load(String yamlFile) throws IOException { return cfg; } - public static MongoSourceConfig load(Map map) throws IOException { - final ObjectMapper mapper = new ObjectMapper(); - final MongoSourceConfig cfg = - mapper.readValue(mapper.writeValueAsString(map), MongoSourceConfig.class); - - return cfg; + public static MongoSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, MongoSourceConfig.class, sourceContext); } /** diff --git a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java index b1166eac5722a..c86e45feb2348 100644 --- a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java +++ b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.io.mongodb; import java.util.Map; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -34,7 +36,27 @@ public void testLoadMapConfig() throws IOException { commonConfigMap.put("batchSize", TestHelper.BATCH_SIZE); commonConfigMap.put("batchTimeMs", TestHelper.BATCH_TIME); - final MongoSinkConfig cfg = MongoSinkConfig.load(commonConfigMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(commonConfigMap, sinkContext); + + assertEquals(cfg.getMongoUri(), TestHelper.URI); + assertEquals(cfg.getDatabase(), TestHelper.DB); + assertEquals(cfg.getCollection(), TestHelper.COLL); + assertEquals(cfg.getBatchSize(), TestHelper.BATCH_SIZE); + assertEquals(cfg.getBatchTimeMs(), TestHelper.BATCH_TIME); + } + + @Test + public void testLoadMapConfigUrlFromSecret() throws IOException { + final Map commonConfigMap = TestHelper.createCommonConfigMap(); + commonConfigMap.put("batchSize", TestHelper.BATCH_SIZE); + commonConfigMap.put("batchTimeMs", TestHelper.BATCH_TIME); + commonConfigMap.remove("mongoUri"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("mongoUri")) + .thenReturn(TestHelper.URI); + final MongoSinkConfig cfg = MongoSinkConfig.load(commonConfigMap, sinkContext); assertEquals(cfg.getMongoUri(), TestHelper.URI); assertEquals(cfg.getDatabase(), TestHelper.DB); @@ -44,12 +66,13 @@ public void testLoadMapConfig() throws IOException { } @Test(expectedExceptions = IllegalArgumentException.class, - expectedExceptionsMessageRegExp = "Required MongoDB URI is not set.") + expectedExceptionsMessageRegExp = "mongoUri cannot be null") public void testBadMongoUri() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeMongoUri(configMap); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -60,7 +83,8 @@ public void testBadDatabase() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeDatabase(configMap); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -71,7 +95,8 @@ public void testBadCollection() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeCollection(configMap); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -82,7 +107,8 @@ public void testBadBatchSize() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchSize(configMap, 0); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -93,7 +119,8 @@ public void testBadBatchTime() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchTime(configMap, 0L); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } diff --git a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java index e7fd01549b033..528cd0237ef16 100644 --- a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java +++ b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java @@ -23,6 +23,8 @@ import java.io.File; import java.io.IOException; import java.util.Map; +import org.apache.pulsar.io.core.SourceContext; +import org.mockito.Mockito; import org.testng.annotations.Test; public class MongoSourceConfigTest { @@ -32,7 +34,27 @@ public void testLoadMapConfig() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putSyncType(configMap, TestHelper.SYNC_TYPE); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); + + assertEquals(cfg.getMongoUri(), TestHelper.URI); + assertEquals(cfg.getDatabase(), TestHelper.DB); + assertEquals(cfg.getCollection(), TestHelper.COLL); + assertEquals(cfg.getSyncType(), TestHelper.SYNC_TYPE); + assertEquals(cfg.getBatchSize(), TestHelper.BATCH_SIZE); + assertEquals(cfg.getBatchTimeMs(), TestHelper.BATCH_TIME); + } + + @Test + public void testLoadMapConfigUriFromSecret() throws IOException { + final Map configMap = TestHelper.createCommonConfigMap(); + TestHelper.putSyncType(configMap, TestHelper.SYNC_TYPE); + configMap.remove("mongoUri"); + + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("mongoUri")) + .thenReturn(TestHelper.URI); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); assertEquals(cfg.getMongoUri(), TestHelper.URI); assertEquals(cfg.getDatabase(), TestHelper.DB); @@ -43,12 +65,13 @@ public void testLoadMapConfig() throws IOException { } @Test(expectedExceptions = IllegalArgumentException.class, - expectedExceptionsMessageRegExp = "Required MongoDB URI is not set.") + expectedExceptionsMessageRegExp = "mongoUri cannot be null") public void testBadMongoUri() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeMongoUri(configMap); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } @@ -61,7 +84,8 @@ public void testBadSyncType() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putSyncType(configMap, "wrong_sync_type_str"); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } @@ -72,7 +96,8 @@ public void testBadBatchSize() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchSize(configMap, 0); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } @@ -83,7 +108,8 @@ public void testBadBatchTime() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchTime(configMap, 0L); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 798ec5e0d084a..33f3dc55b891c 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -32,6 +32,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java index f317a35734e69..89192c42346e8 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java @@ -53,7 +53,7 @@ public class RabbitMQSink implements Sink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - rabbitMQSinkConfig = RabbitMQSinkConfig.load(config); + rabbitMQSinkConfig = RabbitMQSinkConfig.load(config, sinkContext); rabbitMQSinkConfig.validate(); ConnectionFactory connectionFactory = rabbitMQSinkConfig.createConnectionFactory(); diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java index c1f8d6b8ad3d3..39f97e5e460c8 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.google.common.base.Preconditions; import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -28,6 +27,8 @@ import lombok.Data; import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -60,14 +61,12 @@ public static RabbitMQSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), RabbitMQSinkConfig.class); } - public static RabbitMQSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), RabbitMQSinkConfig.class); + public static RabbitMQSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, RabbitMQSinkConfig.class, sinkContext); } @Override public void validate() { super.validate(); - Preconditions.checkNotNull(exchangeName, "exchangeName property not set."); } } diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java index d15108c4d8288..b0b7ef31b08de 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java @@ -54,7 +54,7 @@ public class RabbitMQSource extends PushSource { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - rabbitMQSourceConfig = RabbitMQSourceConfig.load(config); + rabbitMQSourceConfig = RabbitMQSourceConfig.load(config, sourceContext); rabbitMQSourceConfig.validate(); ConnectionFactory connectionFactory = rabbitMQSourceConfig.createConnectionFactory(); diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java index f24018e70da13..01e23a7146080 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java @@ -28,6 +28,8 @@ import lombok.Data; import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -66,9 +68,8 @@ public static RabbitMQSourceConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), RabbitMQSourceConfig.class); } - public static RabbitMQSourceConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), RabbitMQSourceConfig.class); + public static RabbitMQSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, RabbitMQSourceConfig.class, sourceContext); } @Override diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java index 3d4fd6f46e16f..8706cb567524f 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.rabbitmq.sink; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.rabbitmq.RabbitMQSinkConfig; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -71,7 +73,45 @@ public final void loadFromMapTest() throws IOException { map.put("exchangeName", "test-exchange"); map.put("exchangeType", "test-exchange-type"); - RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals(config.getHost(), "localhost"); + assertEquals(config.getPort(), Integer.parseInt("5673")); + assertEquals(config.getVirtualHost(), "/"); + assertEquals(config.getUsername(), "guest"); + assertEquals(config.getPassword(), "guest"); + assertEquals(config.getConnectionName(), "test-connection"); + assertEquals(config.getRequestedChannelMax(), Integer.parseInt("0")); + assertEquals(config.getRequestedFrameMax(), Integer.parseInt("0")); + assertEquals(config.getConnectionTimeout(), Integer.parseInt("60000")); + assertEquals(config.getHandshakeTimeout(), Integer.parseInt("10000")); + assertEquals(config.getRequestedHeartbeat(), Integer.parseInt("60")); + assertEquals(config.getExchangeName(), "test-exchange"); + assertEquals(config.getExchangeType(), "test-exchange-type"); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("host", "localhost"); + map.put("port", "5673"); + map.put("virtualHost", "/"); + map.put("connectionName", "test-connection"); + map.put("requestedChannelMax", "0"); + map.put("requestedFrameMax", "0"); + map.put("connectionTimeout", "60000"); + map.put("handshakeTimeout", "10000"); + map.put("requestedHeartbeat", "60"); + map.put("exchangeName", "test-exchange"); + map.put("exchangeType", "test-exchange-type"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("username")) + .thenReturn("guest"); + Mockito.when(sinkContext.getSecret("password")) + .thenReturn("guest"); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals(config.getHost(), "localhost"); assertEquals(config.getPort(), Integer.parseInt("5673")); @@ -105,12 +145,13 @@ public final void validValidateTest() throws IOException { map.put("exchangeName", "test-exchange"); map.put("exchangeType", "test-exchange-type"); - RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "exchangeName property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "exchangeName cannot be null") public final void missingExchangeValidateTest() throws IOException { Map map = new HashMap<>(); map.put("host", "localhost"); @@ -126,7 +167,8 @@ public final void missingExchangeValidateTest() throws IOException { map.put("requestedHeartbeat", "60"); map.put("exchangeType", "test-exchange-type"); - RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); config.validate(); } diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java index c33e0070c6fd0..43a90062fa453 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.rabbitmq.source; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.rabbitmq.RabbitMQSourceConfig; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -76,7 +78,50 @@ public final void loadFromMapTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "true"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); + assertNotNull(config); + assertEquals("localhost", config.getHost()); + assertEquals(Integer.parseInt("5672"), config.getPort()); + assertEquals("/", config.getVirtualHost()); + assertEquals("guest", config.getUsername()); + assertEquals("guest", config.getPassword()); + assertEquals("test-queue", config.getQueueName()); + assertEquals("test-connection", config.getConnectionName()); + assertEquals(Integer.parseInt("0"), config.getRequestedChannelMax()); + assertEquals(Integer.parseInt("0"), config.getRequestedFrameMax()); + assertEquals(Integer.parseInt("60000"), config.getConnectionTimeout()); + assertEquals(Integer.parseInt("10000"), config.getHandshakeTimeout()); + assertEquals(Integer.parseInt("60"), config.getRequestedHeartbeat()); + assertEquals(Integer.parseInt("0"), config.getPrefetchCount()); + assertEquals(false, config.isPrefetchGlobal()); + assertEquals(false, config.isPrefetchGlobal()); + assertEquals(true, config.isPassive()); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("host", "localhost"); + map.put("port", "5672"); + map.put("virtualHost", "/"); + map.put("queueName", "test-queue"); + map.put("connectionName", "test-connection"); + map.put("requestedChannelMax", "0"); + map.put("requestedFrameMax", "0"); + map.put("connectionTimeout", "60000"); + map.put("handshakeTimeout", "10000"); + map.put("requestedHeartbeat", "60"); + map.put("prefetchCount", "0"); + map.put("prefetchGlobal", "false"); + map.put("passive", "true"); + + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("username")) + .thenReturn("guest"); + Mockito.when(sourceContext.getSecret("password")) + .thenReturn("guest"); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); assertNotNull(config); assertEquals("localhost", config.getHost()); assertEquals(Integer.parseInt("5672"), config.getPort()); @@ -115,12 +160,13 @@ public final void validValidateTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "false"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "host property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "host cannot be null") public final void missingHostValidateTest() throws IOException { Map map = new HashMap<>(); map.put("port", "5672"); @@ -138,7 +184,8 @@ public final void missingHostValidateTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "false"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); config.validate(); } @@ -162,7 +209,8 @@ public final void invalidPrefetchCountTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "false"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); config.validate(); } diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java index abff93a363298..2771185b84162 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java @@ -44,7 +44,7 @@ public void tearDown() { } @Test - public void TestOpenAndWriteSink() { + public void TestOpenAndWriteSink() throws Exception { Map configs = new HashMap<>(); configs.put("host", "localhost"); configs.put("port", "5672"); @@ -67,6 +67,7 @@ public void TestOpenAndWriteSink() { // open should success // rabbitmq service may need time to initialize Awaitility.await().ignoreExceptions().untilAsserted(() -> source.open(configs, null)); + source.close(); } } diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 767e3cd9eab15..67560cc41b451 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -32,6 +32,11 @@ Pulsar IO :: Redis + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java index 978e7de31a51c..89ec684dded72 100644 --- a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java +++ b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java @@ -88,13 +88,11 @@ public class RedisAbstractConfig implements Serializable { @FieldDoc( required = false, - defaultValue = "10000L", + defaultValue = "10000", help = "The amount of time in milliseconds to wait before timing out when connecting") private long connectTimeout = 10000L; public void validate() { - Preconditions.checkNotNull(redisHosts, "redisHosts property not set."); - Preconditions.checkNotNull(redisDatabase, "redisDatabase property not set."); Preconditions.checkNotNull(clientMode, "clientMode property not set."); } @@ -105,7 +103,6 @@ public enum ClientMode { public List getHostAndPorts() { List hostAndPorts = Lists.newArrayList(); - Preconditions.checkNotNull(redisHosts, "redisHosts property not set."); String[] hosts = StringUtils.split(redisHosts, ","); for (String host : hosts) { HostAndPort hostAndPort = HostAndPort.fromString(host); diff --git a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java index bff0a5c2da592..ebd6e9dbab272 100644 --- a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java +++ b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java @@ -68,7 +68,7 @@ public class RedisSink implements Sink { public void open(Map config, SinkContext sinkContext) throws Exception { log.info("Open Redis Sink"); - redisSinkConfig = RedisSinkConfig.load(config); + redisSinkConfig = RedisSinkConfig.load(config, sinkContext); redisSinkConfig.validate(); redisSession = RedisSession.create(redisSinkConfig); diff --git a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java index a9db66812a475..f7a70cb65a826 100644 --- a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java +++ b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java @@ -28,6 +28,8 @@ import lombok.Data; import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; import org.apache.pulsar.io.redis.RedisAbstractConfig; @@ -40,13 +42,13 @@ public class RedisSinkConfig extends RedisAbstractConfig implements Serializable @FieldDoc( required = false, - defaultValue = "10000L", + defaultValue = "10000", help = "The amount of time in milliseconds before an operation is marked as timed out") private long operationTimeout = 10000L; @FieldDoc( required = false, - defaultValue = "1000L", + defaultValue = "1000", help = "The Redis operation time in milliseconds") private long batchTimeMs = 1000L; @@ -62,9 +64,8 @@ public static RedisSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), RedisSinkConfig.class); } - public static RedisSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), RedisSinkConfig.class); + public static RedisSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, RedisSinkConfig.class, sinkContext); } @Override diff --git a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java index 1316d0994a1cd..39fc6e540c242 100644 --- a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java +++ b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.redis.sink; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.redis.RedisAbstractConfig; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -62,7 +64,34 @@ public final void loadFromMapTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals(config.getRedisHosts(), "localhost:6379"); + assertEquals(config.getRedisPassword(), "fake@123"); + assertEquals(config.getRedisDatabase(), Integer.parseInt("1")); + assertEquals(config.getClientMode(), "Standalone"); + assertEquals(config.getOperationTimeout(), Long.parseLong("2000")); + assertEquals(config.getBatchSize(), Integer.parseInt("100")); + assertEquals(config.getBatchTimeMs(), Long.parseLong("1000")); + assertEquals(config.getConnectTimeout(), Long.parseLong("3000")); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap(); + map.put("redisHosts", "localhost:6379"); + map.put("redisDatabase", "1"); + map.put("clientMode", "Standalone"); + map.put("operationTimeout", "2000"); + map.put("batchSize", "100"); + map.put("batchTimeMs", "1000"); + map.put("connectTimeout", "3000"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("redisPassword")) + .thenReturn("fake@123"); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals(config.getRedisHosts(), "localhost:6379"); assertEquals(config.getRedisPassword(), "fake@123"); @@ -86,12 +115,13 @@ public final void validValidateTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "redisHosts property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "redisHosts cannot be null") public final void missingValidValidateTableNameTest() throws IOException { Map map = new HashMap(); map.put("redisPassword", "fake@123"); @@ -102,7 +132,8 @@ public final void missingValidValidateTableNameTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); } @@ -119,7 +150,8 @@ public final void invalidBatchTimeMsTest() throws IOException { map.put("batchTimeMs", "-100"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); } @@ -136,7 +168,8 @@ public final void invalidClientModeTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); RedisAbstractConfig.ClientMode.valueOf(config.getClientMode().toUpperCase()); diff --git a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java index 214151345b42c..2b407fafa5e04 100644 --- a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java +++ b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java @@ -21,7 +21,9 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.SinkRecord; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.redis.EmbeddedRedisUtils; +import org.mockito.Mockito; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -66,7 +68,8 @@ public void TestOpenAndWriteSink() throws Exception { Record record = build("fakeTopic", "fakeKey", "fakeValue"); // open should success - sink.open(configs, null); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + sink.open(configs, sinkContext); // write should success. sink.write(record); diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index b5765b17c5638..3ba603b776012 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -36,6 +36,11 @@ Pulsar IO :: Solr + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.parent.groupId} pulsar-io-core diff --git a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java index de9cdb4a9d82a..202c782c14c49 100644 --- a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java +++ b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java @@ -48,7 +48,7 @@ public abstract class SolrAbstractSink implements Sink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - solrSinkConfig = SolrSinkConfig.load(config); + solrSinkConfig = SolrSinkConfig.load(config, sinkContext); solrSinkConfig.validate(); enableBasicAuth = !Strings.isNullOrEmpty(solrSinkConfig.getUsername()); diff --git a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java index 02733d230bdcb..daa93a366b110 100644 --- a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java +++ b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java @@ -27,6 +27,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -84,9 +86,8 @@ public static SolrSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), SolrSinkConfig.class); } - public static SolrSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), SolrSinkConfig.class); + public static SolrSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, SolrSinkConfig.class, sinkContext); } public void validate() { diff --git a/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java b/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java index 42d2121dbfcbd..2c2447a637d35 100644 --- a/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java +++ b/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.io.solr; import com.google.common.collect.Lists; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -61,7 +63,31 @@ public final void loadFromMapTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals(config.getSolrUrl(), "localhost:2181,localhost:2182/chroot"); + assertEquals(config.getSolrMode(), "SolrCloud"); + assertEquals(config.getSolrCollection(), "techproducts"); + assertEquals(config.getSolrCommitWithinMs(), Integer.parseInt("100")); + assertEquals(config.getUsername(), "fakeuser"); + assertEquals(config.getPassword(), "fake@123"); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("solrUrl", "localhost:2181,localhost:2182/chroot"); + map.put("solrMode", "SolrCloud"); + map.put("solrCollection", "techproducts"); + map.put("solrCommitWithinMs", "100"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("username")) + .thenReturn("fakeuser"); + Mockito.when(sinkContext.getSecret("password")) + .thenReturn("fake@123"); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals(config.getSolrUrl(), "localhost:2181,localhost:2182/chroot"); assertEquals(config.getSolrMode(), "SolrCloud"); @@ -81,12 +107,13 @@ public final void validValidateTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "solrUrl property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "solrUrl cannot be null") public final void missingValidValidateSolrModeTest() throws IOException { Map map = new HashMap<>(); map.put("solrMode", "SolrCloud"); @@ -95,7 +122,8 @@ public final void missingValidValidateSolrModeTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); } @@ -110,7 +138,8 @@ public final void invalidBatchTimeMsTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); } @@ -125,7 +154,8 @@ public final void invalidClientModeTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); SolrAbstractSink.SolrMode.valueOf(config.getSolrMode().toUpperCase()); @@ -141,7 +171,8 @@ public final void validZkChrootTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); String url = config.getSolrUrl(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java index a7fb7192cb5fe..194b0d6a2f8a8 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java @@ -109,9 +109,9 @@ public EtcdMetadataStore(String metadataURL, MetadataStoreConfig conf, boolean e try { this.client = newEtcdClient(metadataURL, conf); this.kv = client.getKVClient(); - this.client.getWatchClient().watch(ByteSequence.from("\0", StandardCharsets.UTF_8), + this.client.getWatchClient().watch(ByteSequence.from("/", StandardCharsets.UTF_8), WatchOption.newBuilder() - .withPrefix(ByteSequence.from("/", StandardCharsets.UTF_8)) + .isPrefix(true) .build(), this::handleWatchResponse); if (enableSessionWatcher) { this.sessionWatcher = @@ -285,7 +285,7 @@ protected void batchOperation(List ops) { .withKeysOnly(true) .withSortField(GetOption.SortTarget.KEY) .withSortOrder(GetOption.SortOrder.ASCEND) - .withPrefix(prefix) + .isPrefix(true) .build())); break; } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index a4cb7926bebf1..7178a0ceda4db 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -373,7 +373,7 @@ public class ProxyConfiguration implements PulsarConfiguration { @FieldContext( category = CATEGORY_HTTP, - doc = "Whether to enable the proxy's /metrics, /proxy-stats, and /status.html http endpoints" + doc = "Whether to enable the proxy's /metrics and /proxy-stats http endpoints" ) private boolean enableProxyStatsEndpoints = true; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java new file mode 100644 index 0000000000000..72b70eaab0805 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.proxy.server; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.mockito.Mockito; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ProxyWithExtensibleLoadManagerTest extends MultiBrokerBaseTest { + + private static final int TEST_TIMEOUT_MS = 30_000; + + private ProxyService proxyService; + + @Override + public int numberOfAdditionalBrokers() { + return 1; + } + + @Override + public void doInitConf() throws Exception { + super.doInitConf(); + configureExtensibleLoadManager(conf); + } + + @Override + protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) { + return configureExtensibleLoadManager(getDefaultConf()); + } + + private ServiceConfiguration configureExtensibleLoadManager(ServiceConfiguration config) { + config.setNumIOThreads(8); + config.setLoadBalancerInFlightServiceUnitStateWaitingTimeInMillis(5 * 1000); + config.setLoadBalancerServiceUnitStateMonitorIntervalInSeconds(1); + config.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + config.setLoadBalancerSheddingEnabled(false); + return config; + } + + private ProxyConfiguration initializeProxyConfig() { + var proxyConfig = new ProxyConfiguration(); + proxyConfig.setNumIOThreads(8); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + return proxyConfig; + } + + private LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { + LookupService svc = (LookupService) FieldUtils.readDeclaredField(client, "lookup", true); + var lookup = spy(svc); + FieldUtils.writeDeclaredField(client, "lookup", lookup, true); + return lookup; + } + + private PulsarClientImpl createClient(ProxyService proxyService) { + try { + return Mockito.spy((PulsarClientImpl) PulsarClient.builder(). + serviceUrl(proxyService.getServiceUrl()). + build()); + } catch (PulsarClientException e) { + throw new CompletionException(e); + } + } + + @BeforeMethod(alwaysRun = true) + public void proxySetup() throws Exception { + var proxyConfig = initializeProxyConfig(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig)))); + doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); + doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) + .createConfigurationMetadataStore(); + proxyService.start(); + } + + @AfterMethod(alwaysRun = true) + public void proxyCleanup() throws Exception { + if (proxyService != null) { + proxyService.close(); + } + } + + @Test(timeOut = TEST_TIMEOUT_MS, invocationCount = 100, skipFailedInvocations = true) + public void testProxyProduceConsume() throws Exception { + var namespaceName = NamespaceName.get("public", "default"); + var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, + BrokerTestUtil.newUniqueName("testProxyProduceConsume")); + + @Cleanup("shutdownNow") + var threadPool = Executors.newCachedThreadPool(); + + var producerClientFuture = CompletableFuture.supplyAsync(() -> createClient(proxyService), threadPool); + var consumerClientFuture = CompletableFuture.supplyAsync(() -> createClient(proxyService), threadPool); + + @Cleanup + var producerClient = producerClientFuture.get(); + @Cleanup + var producer = producerClient.newProducer(Schema.INT32).topic(topicName.toString()).create(); + var producerLookupServiceSpy = spyLookupService(producerClient); + + @Cleanup + var consumerClient = consumerClientFuture.get(); + @Cleanup + var consumer = consumerClient.newConsumer(Schema.INT32).topic(topicName.toString()). + subscriptionInitialPosition(SubscriptionInitialPosition.Earliest). + subscriptionName(BrokerTestUtil.newUniqueName("my-sub")). + ackTimeout(1000, TimeUnit.MILLISECONDS). + subscribe(); + var consumerLookupServiceSpy = spyLookupService(consumerClient); + + var bundleRange = admin.lookups().getBundleRange(topicName.toString()); + + var cdl = new CountDownLatch(1); + var semSend = new Semaphore(0); + var messagesBeforeUnload = 100; + var messagesAfterUnload = 100; + + var pendingMessageIds = Collections.synchronizedSet(new HashSet()); + var producerFuture = CompletableFuture.runAsync(() -> { + try { + for (int i = 0; i < messagesBeforeUnload + messagesAfterUnload; i++) { + semSend.acquire(); + pendingMessageIds.add(i); + producer.send(i); + } + } catch (Exception e) { + throw new CompletionException(e); + } + }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); + + var consumerFuture = CompletableFuture.runAsync(() -> { + while (!producerFuture.isDone() || !pendingMessageIds.isEmpty()) { + try { + var recvMessage = consumer.receive(1_500, TimeUnit.MILLISECONDS); + if (recvMessage != null) { + consumer.acknowledge(recvMessage); + pendingMessageIds.remove(recvMessage.getValue()); + } + } catch (PulsarClientException e) { + // Retry + } + } + }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); + + var unloadFuture = CompletableFuture.runAsync(() -> { + try { + cdl.await(); + var srcBrokerUrl = admin.lookups().lookupTopic(topicName.toString()); + var dstBrokerUrl = getAllBrokers().stream(). + filter(pulsarService -> !Objects.equals(srcBrokerUrl, pulsarService.getBrokerServiceUrl())). + map(PulsarService::getLookupServiceAddress). + findAny().orElseThrow(() -> new Exception("Could not determine destination broker URL")); + semSend.release(messagesBeforeUnload); + admin.namespaces().unloadNamespaceBundle(namespaceName.toString(), bundleRange, dstBrokerUrl); + semSend.release(messagesAfterUnload); + } catch (Exception e) { + throw new CompletionException(e); + } + }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); + + cdl.countDown(); + + // Verify all futures completed successfully. + unloadFuture.get(); + producerFuture.get(); + consumerFuture.get(); + + verify(producerClient, times(1)).getProxiedConnection(any(), anyInt()); + verify(producerLookupServiceSpy, never()).getBroker(topicName); + + verify(consumerClient, times(1)).getProxiedConnection(any(), anyInt()); + verify(consumerLookupServiceSpy, never()).getBroker(topicName); + } +} diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 80266a2670896..ec301059d87c0 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,21 +231,21 @@ The Apache Software License, Version 2.0 - commons-compress-1.21.jar - commons-lang3-3.11.jar * Netty - - netty-buffer-4.1.100.Final.jar - - netty-codec-4.1.100.Final.jar - - netty-codec-dns-4.1.100.Final.jar - - netty-codec-http-4.1.100.Final.jar - - netty-codec-haproxy-4.1.100.Final.jar - - netty-codec-socks-4.1.100.Final.jar - - netty-handler-proxy-4.1.100.Final.jar - - netty-common-4.1.100.Final.jar - - netty-handler-4.1.100.Final.jar + - netty-buffer-4.1.104.Final.jar + - netty-codec-4.1.104.Final.jar + - netty-codec-dns-4.1.104.Final.jar + - netty-codec-http-4.1.104.Final.jar + - netty-codec-haproxy-4.1.104.Final.jar + - netty-codec-socks-4.1.104.Final.jar + - netty-handler-proxy-4.1.104.Final.jar + - netty-common-4.1.104.Final.jar + - netty-handler-4.1.104.Final.jar - netty-reactive-streams-2.0.6.jar - - netty-resolver-4.1.100.Final.jar - - netty-resolver-dns-4.1.100.Final.jar - - netty-resolver-dns-classes-macos-4.1.100.Final.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar + - netty-resolver-4.1.104.Final.jar + - netty-resolver-dns-4.1.104.Final.jar + - netty-resolver-dns-classes-macos-4.1.104.Final.jar + - netty-resolver-dns-native-macos-4.1.104.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.104.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -253,15 +253,15 @@ The Apache Software License, Version 2.0 - netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - netty-tcnative-classes-2.0.61.Final.jar - - netty-transport-4.1.100.Final.jar - - netty-transport-classes-epoll-4.1.100.Final.jar - - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.100.Final.jar - - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - - netty-codec-http2-4.1.100.Final.jar - - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar + - netty-transport-4.1.104.Final.jar + - netty-transport-classes-epoll-4.1.104.Final.jar + - netty-transport-native-epoll-4.1.104.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.104.Final.jar + - netty-transport-native-unix-common-4.1.104.Final-linux-x86_64.jar + - netty-codec-http2-4.1.104.Final.jar + - netty-incubator-transport-classes-io_uring-0.0.24.Final.jar + - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar + - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar * GRPC - grpc-api-1.55.3.jar - grpc-context-1.55.3.jar diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java index 79efbeba3bc26..e967ba9e51769 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java @@ -19,6 +19,7 @@ package org.apache.pulsar.testclient; import static org.apache.pulsar.broker.resources.LoadBalanceResources.BUNDLE_DATA_BASE_PATH; +import static org.apache.pulsar.broker.resources.LoadBalanceResources.RESOURCE_QUOTA_BASE_PATH; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.beust.jcommander.ParameterException; @@ -61,7 +62,6 @@ */ public class LoadSimulationController { private static final Logger log = LoggerFactory.getLogger(LoadSimulationController.class); - private static final String QUOTA_ROOT = "/loadbalance/resource-quota"; // Input streams for each client to send commands through. private final DataInputStream[] inputStreams; @@ -398,7 +398,7 @@ private void handleCopy(final ShellArguments arguments) throws Exception { for (int i = 0; i < clients.length; ++i) { threadLocalMaps[i] = new HashMap<>(); } - getResourceQuotas(QUOTA_ROOT, sourceZKClient, threadLocalMaps); + getResourceQuotas(RESOURCE_QUOTA_BASE_PATH, sourceZKClient, threadLocalMaps); final List futures = new ArrayList<>(clients.length); int i = 0; log.info("Copying..."); @@ -411,7 +411,7 @@ private void handleCopy(final ShellArguments arguments) throws Exception { // Simulation will send messages in and out at about the same rate, so just make the rate the // average of in and out. - final int tenantStart = QUOTA_ROOT.length() + 1; + final int tenantStart = RESOURCE_QUOTA_BASE_PATH.length() + 1; final int clusterStart = bundle.indexOf('/', tenantStart) + 1; final String sourceTenant = bundle.substring(tenantStart, clusterStart - 1); final int namespaceStart = bundle.indexOf('/', clusterStart) + 1; @@ -424,7 +424,7 @@ private void handleCopy(final ShellArguments arguments) throws Exception { final String mangledNamespace = String.format("%s-%s", manglePrefix, namespace); final BundleData bundleData = initializeBundleData(quota, arguments); final String oldAPITargetPath = String.format( - "/loadbalance/resource-quota/namespace/%s/%s/%s/0x00000000_0xffffffff", tenantName, + "%s/namespace/%s/%s/%s/0x00000000_0xffffffff", BUNDLE_DATA_BASE_PATH, tenantName, cluster, mangledNamespace); final String newAPITargetPath = String.format( "%s/%s/%s/%s/0x00000000_0xffffffff", BUNDLE_DATA_BASE_PATH, tenantName, cluster, @@ -475,7 +475,7 @@ private void handleSimulate(final ShellArguments arguments) throws Exception { for (int i = 0; i < clients.length; ++i) { threadLocalMaps[i] = new HashMap<>(); } - getResourceQuotas(QUOTA_ROOT, zkClient, threadLocalMaps); + getResourceQuotas(RESOURCE_QUOTA_BASE_PATH, zkClient, threadLocalMaps); final List futures = new ArrayList<>(clients.length); int i = 0; log.info("Simulating..."); @@ -484,9 +484,9 @@ private void handleSimulate(final ShellArguments arguments) throws Exception { futures.add(threadPool.submit(() -> { for (final Map.Entry entry : bundleToQuota.entrySet()) { final String bundle = entry.getKey(); - final String newAPIPath = bundle.replace(QUOTA_ROOT, BUNDLE_DATA_BASE_PATH); + final String newAPIPath = bundle.replace(RESOURCE_QUOTA_BASE_PATH, BUNDLE_DATA_BASE_PATH); final ResourceQuota quota = entry.getValue(); - final int tenantStart = QUOTA_ROOT.length() + 1; + final int tenantStart = RESOURCE_QUOTA_BASE_PATH.length() + 1; final String topic = String.format("persistent://%s/t", bundle.substring(tenantStart)); final BundleData bundleData = initializeBundleData(quota, arguments); // Put the bundle data in the new ZooKeeper. diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java index 9bd74be3aa859..bcdb981e1ebe5 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java @@ -505,6 +505,14 @@ public static void main(String[] args) throws Exception { reportHistogram.reset(); oldTime = now; + + if (arguments.testTime > 0) { + if (now > testEndTime) { + log.info("------------------- DONE -----------------------"); + PerfClientUtils.exit(0); + thread.interrupt(); + } + } } pulsarClient.close(); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java index 16a32fe6344f0..a5adb508a18b8 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java @@ -568,7 +568,7 @@ private static void runProducer(int producerId, } } // Send messages on all topics/producers - long totalSent = 0; + AtomicLong totalSent = new AtomicLong(0); AtomicLong numMessageSend = new AtomicLong(0); Semaphore numMsgPerTxnLimit = new Semaphore(arguments.numMessagesPerTransaction); while (true) { @@ -587,7 +587,7 @@ private static void runProducer(int producerId, } if (numMessages > 0) { - if (totalSent++ >= numMessages) { + if (totalSent.get() >= numMessages) { log.info("------------- DONE (reached the maximum number: {} of production) --------------" , numMessages); doneLatch.countDown(); @@ -604,7 +604,7 @@ private static void runProducer(int producerId, if (arguments.payloadFilename != null) { if (messageFormatter != null) { - payloadData = messageFormatter.formatMessage(arguments.producerName, totalSent, + payloadData = messageFormatter.formatMessage(arguments.producerName, totalSent.get(), payloadByteList.get(ThreadLocalRandom.current().nextInt(payloadByteList.size()))); } else { payloadData = payloadByteList.get( @@ -642,13 +642,13 @@ private static void runProducer(int producerId, if (msgKeyMode == MessageKeyGenerationMode.random) { messageBuilder.key(String.valueOf(ThreadLocalRandom.current().nextInt())); } else if (msgKeyMode == MessageKeyGenerationMode.autoIncrement) { - messageBuilder.key(String.valueOf(totalSent)); + messageBuilder.key(String.valueOf(totalSent.get())); } PulsarClient pulsarClient = client; messageBuilder.sendAsync().thenRun(() -> { bytesSent.add(payloadData.length); messagesSent.increment(); - + totalSent.incrementAndGet(); totalMessagesSent.increment(); totalBytesSent.add(payloadData.length); diff --git a/tests/docker-images/java-test-image/Dockerfile b/tests/docker-images/java-test-image/Dockerfile index 6a9c7d10331be..5a1bbf15e93b5 100644 --- a/tests/docker-images/java-test-image/Dockerfile +++ b/tests/docker-images/java-test-image/Dockerfile @@ -39,9 +39,10 @@ ARG JDK_MAJOR_VERSION=17 RUN sed -i -e "s|http://archive\.ubuntu\.com/ubuntu/|${UBUNTU_MIRROR:-http://archive.ubuntu.com/ubuntu/}|g" \ -e "s|http://security\.ubuntu\.com/ubuntu/|${UBUNTU_SECURITY_MIRROR:-http://security.ubuntu.com/ubuntu/}|g" /etc/apt/sources.list \ + && echo 'Acquire::http::Timeout "30";\nAcquire::http::ConnectionAttemptDelayMsec "2000";\nAcquire::https::Timeout "30";\nAcquire::https::ConnectionAttemptDelayMsec "2000";\nAcquire::ftp::Timeout "30";\nAcquire::ftp::ConnectionAttemptDelayMsec "2000";\nAcquire::Retries "15";' > /etc/apt/apt.conf.d/99timeout_and_retries \ && apt-get update \ && apt-get -y dist-upgrade \ - && apt-get -y install wget apt-transport-https + && apt-get -y install ca-certificates wget apt-transport-https # Install Eclipse Temurin Package RUN mkdir -p /etc/apt/keyrings \ diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 99672773dcbc8..602f917700b65 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -40,10 +40,6 @@ FROM apachepulsar/pulsar:latest # However, any processes exec'ing into the containers will run as root, by default. USER root -# We need to define the user in order for supervisord to work correctly -# We don't need a user defined in the public docker image, though. -RUN adduser -u 10000 --gid 0 --disabled-login --disabled-password --gecos '' pulsar - RUN rm -rf /var/lib/apt/lists/* && apt update RUN apt-get clean && apt-get update && apt-get install -y supervisor vim procps curl diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java index 45f4ab5343557..8472ed3db2c2b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java @@ -68,15 +68,22 @@ public class PulsarStateTest extends PulsarStandaloneTestSuite { @Test(groups = {"python_state", "state", "function", "python_function"}) public void testPythonWordCountFunction() throws Exception { + String functionName = "test-wordcount-py-fn-" + randomName(8); + doTestPythonWordCountFunction(functionName); + + // after a function is deleted, its state should be clean + // we just recreate and test the word count function again, and it should have same result + doTestPythonWordCountFunction(functionName); + } + + private void doTestPythonWordCountFunction(String functionName) throws Exception { String inputTopicName = "test-wordcount-py-input-" + randomName(8); String outputTopicName = "test-wordcount-py-output-" + randomName(8); - String functionName = "test-wordcount-py-fn-" + randomName(8); final int numMessages = 10; - // submit the exclamation function submitExclamationFunction( - Runtime.PYTHON, inputTopicName, outputTopicName, functionName); + Runtime.PYTHON, inputTopicName, outputTopicName, functionName); // get function info getFunctionInfoSuccess(functionName); @@ -94,6 +101,15 @@ public void testPythonWordCountFunction() throws Exception { queryState(functionName, "message-" + i, 1); } + // test put state + String state = "{\"key\":\"test-string\",\"stringValue\":\"test value\"}"; + String expect = "\"stringValue\": \"test value\""; + putAndQueryState(functionName, "test-string", state, expect); + + String numberState = "{\"key\":\"test-number\",\"numberValue\":20}"; + String expectNumber = "\"numberValue\": 20"; + putAndQueryState(functionName, "test-number", numberState, expectNumber); + // delete function deleteFunction(functionName); @@ -448,6 +464,30 @@ private void queryState(String functionName, String key, int amount) assertTrue(result.getStdout().contains("\"numberValue\": " + amount)); } + private void putAndQueryState(String functionName, String key, String state, String expect) + throws Exception { + container.execCmd( + PulsarCluster.ADMIN_SCRIPT, + "functions", + "putstate", + "--tenant", "public", + "--namespace", "default", + "--name", functionName, + "--state", state + ); + + ContainerExecResult result = container.execCmd( + PulsarCluster.ADMIN_SCRIPT, + "functions", + "querystate", + "--tenant", "public", + "--namespace", "default", + "--name", functionName, + "--key", key + ); + assertTrue(result.getStdout().contains(expect)); + } + private void publishAndConsumeMessages(String inputTopic, String outputTopic, int numMessages) throws Exception { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index 158827ca34db6..bbb92f6a6cf4c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -316,23 +316,40 @@ public void testIsolationPolicy() throws Exception { parameters1.put("min_limit", "1"); parameters1.put("usage_threshold", "100"); - List activeBrokers = admin.brokers().getActiveBrokers(); + Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted( + () -> { + List activeBrokers = admin.brokers().getActiveBrokers(); + assertEquals(activeBrokers.size(), NUM_BROKERS); + } + ); + try { + admin.namespaces().createNamespace(isolationEnabledNameSpace); + } catch (PulsarAdminException.ConflictException e) { + //expected when retried + } - assertEquals(activeBrokers.size(), NUM_BROKERS); + try { + admin.clusters() + .createNamespaceIsolationPolicy(clusterName, namespaceIsolationPolicyName, NamespaceIsolationData + .builder() + .namespaces(List.of(isolationEnabledNameSpace)) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .primary(List.of(getHostName(0))) + .secondary(List.of(getHostName(1))) + .build()); + } catch (PulsarAdminException.ConflictException e) { + //expected when retried + } - admin.namespaces().createNamespace(isolationEnabledNameSpace); - admin.clusters().createNamespaceIsolationPolicy(clusterName, namespaceIsolationPolicyName, NamespaceIsolationData - .builder() - .namespaces(List.of(isolationEnabledNameSpace)) - .autoFailoverPolicy(AutoFailoverPolicyData.builder() - .policyType(AutoFailoverPolicyType.min_available) - .parameters(parameters1) - .build()) - .primary(List.of(getHostName(0))) - .secondary(List.of(getHostName(1))) - .build()); final String topic = "persistent://" + isolationEnabledNameSpace + "/topic"; - admin.topics().createNonPartitionedTopic(topic); + try { + admin.topics().createNonPartitionedTopic(topic); + } catch (PulsarAdminException.ConflictException e) { + //expected when retried + } String broker = admin.lookups().lookupTopic(topic); From 74ed2cfffd24b8fb0cc7fc516794081a72a472e5 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 08:13:30 -0800 Subject: [PATCH 2/8] Rename getProxiedConnection -> getProxyConnection --- .../java/org/apache/pulsar/client/impl/ConnectionHandler.java | 2 +- .../java/org/apache/pulsar/client/impl/PulsarClientImpl.java | 4 ++-- .../proxy/server/ProxyWithExtensibleLoadManagerTest.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index bee3121d7f689..b63ba0471231e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -99,7 +99,7 @@ protected void grabCnx(Optional hostURI) { URI uri = hostURI.get(); InetSocketAddress address = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); if (useProxy) { - cnxFuture = state.client.getProxiedConnection(address, randomKeyForSelectConnection); + cnxFuture = state.client.getProxyConnection(address, randomKeyForSelectConnection); } else { cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 3b785fdaf2354..179996f4ea9f1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -990,8 +990,8 @@ public CompletableFuture getConnectionToServiceUrl() { return getConnection(address, address, cnxPool.genRandomKeyToSelectCon()); } - public CompletableFuture getProxiedConnection(final InetSocketAddress logicalAddress, - final int randomKeyForSelectConnection) { + public CompletableFuture getProxyConnection(final InetSocketAddress logicalAddress, + final int randomKeyForSelectConnection) { if (!(lookup instanceof BinaryProtoLookupService)) { return FutureUtil.failedFuture(new PulsarClientException.InvalidServiceURL( "Cannot proxy connection through HTTP service URL", null)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index 72b70eaab0805..a4f264abf051c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -220,10 +220,10 @@ public void testProxyProduceConsume() throws Exception { producerFuture.get(); consumerFuture.get(); - verify(producerClient, times(1)).getProxiedConnection(any(), anyInt()); + verify(producerClient, times(1)).getProxyConnection(any(), anyInt()); verify(producerLookupServiceSpy, never()).getBroker(topicName); - verify(consumerClient, times(1)).getProxiedConnection(any(), anyInt()); + verify(consumerClient, times(1)).getProxyConnection(any(), anyInt()); verify(consumerLookupServiceSpy, never()).getBroker(topicName); } } From 9596b049ed926bb3abe829dbfb18579635ce6860 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 08:17:25 -0800 Subject: [PATCH 3/8] Add comment to HttpLookupService regarding isUseProxy --- .../java/org/apache/pulsar/client/impl/HttpLookupService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 33f90271826d2..02d0d10626fa6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -100,7 +100,8 @@ public CompletableFuture getBroker(TopicName topicName) { } InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); - return CompletableFuture.completedFuture(new LookupTopicResult(brokerAddress, brokerAddress, false)); + return CompletableFuture.completedFuture(new LookupTopicResult(brokerAddress, brokerAddress, + false /* HTTP lookups never use the proxy */)); } catch (Exception e) { // Failed to parse url log.warn("[{}] Lookup Failed due to invalid url {}, {}", topicName, uri, e.getMessage()); From 57567b8b41fef6df6f3a74ebed23cdd6b33d92e8 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 08:33:28 -0800 Subject: [PATCH 4/8] Ensure useProxy is set before using in ConnectionHandler --- .../java/org/apache/pulsar/client/impl/ConnectionHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index b63ba0471231e..7700596dca3e8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -47,7 +47,7 @@ public class ConnectionHandler { private final AtomicBoolean duringConnect = new AtomicBoolean(false); protected final int randomKeyForSelectConnection; - private boolean useProxy = false; + private volatile Boolean useProxy; interface Connection { @@ -95,7 +95,7 @@ protected void grabCnx(Optional hostURI) { try { CompletableFuture cnxFuture; - if (hostURI.isPresent()) { + if (hostURI.isPresent() && useProxy != null) { URI uri = hostURI.get(); InetSocketAddress address = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); if (useProxy) { From 9dc33e71e9fee7058abb903cd53b8aa60f338896 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 09:29:28 -0800 Subject: [PATCH 5/8] Revert invocationCount=100 on testProxyProduceConsume --- .../pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index a4f264abf051c..72d11d0ee6a1c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -135,7 +135,7 @@ public void proxyCleanup() throws Exception { } } - @Test(timeOut = TEST_TIMEOUT_MS, invocationCount = 100, skipFailedInvocations = true) + @Test(timeOut = TEST_TIMEOUT_MS) public void testProxyProduceConsume() throws Exception { var namespaceName = NamespaceName.get("public", "default"); var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, From e7bf8b549db7263f4011b49102f368f9ab353ac1 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 13:27:33 -0800 Subject: [PATCH 6/8] Add proxy reconnect test --- .../ProxyWithExtensibleLoadManagerTest.java | 150 ++++++++++++++++-- 1 file changed, 136 insertions(+), 14 deletions(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index 72d11d0ee6a1c..b6e3b6d7a3823 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -25,6 +25,9 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.HashSet; import java.util.Objects; @@ -36,6 +39,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.MultiBrokerBaseTest; @@ -43,22 +47,29 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.ServiceNameResolver; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.net.ServiceURI; import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.jetbrains.annotations.NotNull; import org.mockito.Mockito; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +@Slf4j public class ProxyWithExtensibleLoadManagerTest extends MultiBrokerBaseTest { private static final int TEST_TIMEOUT_MS = 30_000; @@ -100,16 +111,19 @@ private ProxyConfiguration initializeProxyConfig() { return proxyConfig; } - private LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { - LookupService svc = (LookupService) FieldUtils.readDeclaredField(client, "lookup", true); - var lookup = spy(svc); - FieldUtils.writeDeclaredField(client, "lookup", lookup, true); - return lookup; + private T spyField(Object target, String fieldName) throws IllegalAccessException { + T t = (T) FieldUtils.readDeclaredField(target, fieldName, true); + var fieldSpy = spy(t); + FieldUtils.writeDeclaredField(target, fieldName, fieldSpy, true); + return fieldSpy; } private PulsarClientImpl createClient(ProxyService proxyService) { try { return Mockito.spy((PulsarClientImpl) PulsarClient.builder(). + // operationTimeout(5, TimeUnit.SECONDS). + // connectionTimeout(5, TimeUnit.SECONDS). + // lookupTimeout(5, TimeUnit.SECONDS). serviceUrl(proxyService.getServiceUrl()). build()); } catch (PulsarClientException e) { @@ -117,6 +131,22 @@ private PulsarClientImpl createClient(ProxyService proxyService) { } } + @NotNull + private InetSocketAddress getSourceBrokerInetAddress(TopicName topicName) throws PulsarAdminException { + var srcBrokerUrl = admin.lookups().lookupTopic(topicName.toString()); + var serviceUri = ServiceURI.create(srcBrokerUrl); + var uri = serviceUri.getUri(); + return InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); + } + + private String getDstBrokerLookupUrl(TopicName topicName) throws Exception { + var srcBrokerUrl = admin.lookups().lookupTopic(topicName.toString()); + return getAllBrokers().stream(). + filter(pulsarService -> !Objects.equals(srcBrokerUrl, pulsarService.getBrokerServiceUrl())). + map(PulsarService::getLookupServiceAddress). + findAny().orElseThrow(() -> new Exception("Could not determine destination broker lookup URL")); + } + @BeforeMethod(alwaysRun = true) public void proxySetup() throws Exception { var proxyConfig = initializeProxyConfig(); @@ -135,7 +165,7 @@ public void proxyCleanup() throws Exception { } } - @Test(timeOut = TEST_TIMEOUT_MS) + @Test(timeOut = TEST_TIMEOUT_MS, invocationCount = 100) public void testProxyProduceConsume() throws Exception { var namespaceName = NamespaceName.get("public", "default"); var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, @@ -151,7 +181,7 @@ public void testProxyProduceConsume() throws Exception { var producerClient = producerClientFuture.get(); @Cleanup var producer = producerClient.newProducer(Schema.INT32).topic(topicName.toString()).create(); - var producerLookupServiceSpy = spyLookupService(producerClient); + LookupService producerLookupServiceSpy = spyField(producerClient, "lookup"); @Cleanup var consumerClient = consumerClientFuture.get(); @@ -161,7 +191,7 @@ public void testProxyProduceConsume() throws Exception { subscriptionName(BrokerTestUtil.newUniqueName("my-sub")). ackTimeout(1000, TimeUnit.MILLISECONDS). subscribe(); - var consumerLookupServiceSpy = spyLookupService(consumerClient); + LookupService consumerLookupServiceSpy = spyField(consumerClient, "lookup"); var bundleRange = admin.lookups().getBundleRange(topicName.toString()); @@ -200,13 +230,9 @@ public void testProxyProduceConsume() throws Exception { var unloadFuture = CompletableFuture.runAsync(() -> { try { cdl.await(); - var srcBrokerUrl = admin.lookups().lookupTopic(topicName.toString()); - var dstBrokerUrl = getAllBrokers().stream(). - filter(pulsarService -> !Objects.equals(srcBrokerUrl, pulsarService.getBrokerServiceUrl())). - map(PulsarService::getLookupServiceAddress). - findAny().orElseThrow(() -> new Exception("Could not determine destination broker URL")); + var dstBrokerLookupUrl = getDstBrokerLookupUrl(topicName); semSend.release(messagesBeforeUnload); - admin.namespaces().unloadNamespaceBundle(namespaceName.toString(), bundleRange, dstBrokerUrl); + admin.namespaces().unloadNamespaceBundle(namespaceName.toString(), bundleRange, dstBrokerLookupUrl); semSend.release(messagesAfterUnload); } catch (Exception e) { throw new CompletionException(e); @@ -226,4 +252,100 @@ public void testProxyProduceConsume() throws Exception { verify(consumerClient, times(1)).getProxyConnection(any(), anyInt()); verify(consumerLookupServiceSpy, never()).getBroker(topicName); } + + @Test(timeOut = TEST_TIMEOUT_MS, invocationCount = 100) + public void testClientReconnectsToBrokerOnProxyClosing() throws Exception { + var namespaceName = NamespaceName.get("public", "default"); + var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, + BrokerTestUtil.newUniqueName("testProxyReconnect")); + + @Cleanup("shutdownNow") + var threadPool = Executors.newCachedThreadPool(); + + var producerClientFuture = CompletableFuture.supplyAsync(() -> createClient(proxyService), threadPool); + var consumerClientFuture = CompletableFuture.supplyAsync(() -> createClient(proxyService), threadPool); + + @Cleanup + var producerClient = producerClientFuture.get(); + @Cleanup + var producer = (ProducerImpl) producerClient.newProducer(Schema.INT32).topic(topicName.toString()). + create(); + LookupService producerLookupServiceSpy = spyField(producerClient, "lookup"); + when(((ServiceNameResolver) spyField(producerLookupServiceSpy, "serviceNameResolver")).resolveHost()). + thenCallRealMethod().then(invocation -> getSourceBrokerInetAddress(topicName)); + + @Cleanup + var consumerClient = consumerClientFuture.get(); + @Cleanup + var consumer = (ConsumerImpl) consumerClient.newConsumer(Schema.INT32).topic(topicName.toString()). + subscriptionInitialPosition(SubscriptionInitialPosition.Earliest). + subscriptionName(BrokerTestUtil.newUniqueName("my-sub")). + ackTimeout(1000, TimeUnit.MILLISECONDS). + subscribe(); + LookupService consumerLookupServiceSpy = spyField(consumerClient, "lookup"); + when(((ServiceNameResolver) spyField(consumerLookupServiceSpy, "serviceNameResolver")).resolveHost()). + thenCallRealMethod().then(invocation -> getSourceBrokerInetAddress(topicName)); + + var bundleRange = admin.lookups().getBundleRange(topicName.toString()); + + var semSend = new Semaphore(0); + var messagesPerPhase = 100; + var phases = 4; + var totalMessages = messagesPerPhase * phases; + var cdlSentMessages = new CountDownLatch(messagesPerPhase * 2); + + var pendingMessageIds = Collections.synchronizedSet(new HashSet()); + var producerFuture = CompletableFuture.runAsync(() -> { + try { + for (int i = 0; i < totalMessages; i++) { + semSend.acquire(); + pendingMessageIds.add(i); + producer.send(i); + cdlSentMessages.countDown(); + } + } catch (Exception e) { + throw new CompletionException(e); + } + }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); + + var consumerFuture = CompletableFuture.runAsync(() -> { + while (!producerFuture.isDone() || !pendingMessageIds.isEmpty()) { + try { + var recvMessage = consumer.receive(1_500, TimeUnit.MILLISECONDS); + if (recvMessage != null) { + consumer.acknowledge(recvMessage); + pendingMessageIds.remove(recvMessage.getValue()); + } + } catch (PulsarClientException e) { + // Retry + } + } + }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); + + var dstBrokerLookupUrl = getDstBrokerLookupUrl(topicName); + semSend.release(messagesPerPhase); + admin.namespaces().unloadNamespaceBundle(namespaceName.toString(), bundleRange, dstBrokerLookupUrl); + semSend.release(messagesPerPhase); + + cdlSentMessages.await(); + assertEquals(FieldUtils.readDeclaredField(producer.getConnectionHandler(), "useProxy", true), Boolean.TRUE); + assertEquals(FieldUtils.readDeclaredField(consumer.getConnectionHandler(), "useProxy", true), Boolean.TRUE); + semSend.release(messagesPerPhase); + proxyService.close(); + proxyService = null; + semSend.release(messagesPerPhase); + + // Verify produce/consume futures completed successfully. + producerFuture.get(); + consumerFuture.get(); + + assertEquals(FieldUtils.readDeclaredField(producer.getConnectionHandler(), "useProxy", true), Boolean.FALSE); + assertEquals(FieldUtils.readDeclaredField(consumer.getConnectionHandler(), "useProxy", true), Boolean.FALSE); + + verify(producerClient, times(1)).getProxyConnection(any(), anyInt()); + verify(producerLookupServiceSpy, times(1)).getBroker(topicName); + + verify(consumerClient, times(1)).getProxyConnection(any(), anyInt()); + verify(consumerLookupServiceSpy, times(1)).getBroker(topicName); + } } From 78bfea5c3a6cb25d3885a0af012a017754d30aee Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 13:35:33 -0800 Subject: [PATCH 7/8] Cosmetic fixes --- .../ProxyWithExtensibleLoadManagerTest.java | 24 ++++--------------- 1 file changed, 5 insertions(+), 19 deletions(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index b6e3b6d7a3823..b0668f5ea07d2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -121,9 +121,6 @@ private T spyField(Object target, String fieldName) throws IllegalAccessExce private PulsarClientImpl createClient(ProxyService proxyService) { try { return Mockito.spy((PulsarClientImpl) PulsarClient.builder(). - // operationTimeout(5, TimeUnit.SECONDS). - // connectionTimeout(5, TimeUnit.SECONDS). - // lookupTimeout(5, TimeUnit.SECONDS). serviceUrl(proxyService.getServiceUrl()). build()); } catch (PulsarClientException e) { @@ -195,7 +192,6 @@ public void testProxyProduceConsume() throws Exception { var bundleRange = admin.lookups().getBundleRange(topicName.toString()); - var cdl = new CountDownLatch(1); var semSend = new Semaphore(0); var messagesBeforeUnload = 100; var messagesAfterUnload = 100; @@ -227,22 +223,12 @@ public void testProxyProduceConsume() throws Exception { } }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); - var unloadFuture = CompletableFuture.runAsync(() -> { - try { - cdl.await(); - var dstBrokerLookupUrl = getDstBrokerLookupUrl(topicName); - semSend.release(messagesBeforeUnload); - admin.namespaces().unloadNamespaceBundle(namespaceName.toString(), bundleRange, dstBrokerLookupUrl); - semSend.release(messagesAfterUnload); - } catch (Exception e) { - throw new CompletionException(e); - } - }, threadPool).orTimeout(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); - - cdl.countDown(); + var dstBrokerLookupUrl = getDstBrokerLookupUrl(topicName); + semSend.release(messagesBeforeUnload); + admin.namespaces().unloadNamespaceBundle(namespaceName.toString(), bundleRange, dstBrokerLookupUrl); + semSend.release(messagesAfterUnload); // Verify all futures completed successfully. - unloadFuture.get(); producerFuture.get(); consumerFuture.get(); @@ -257,7 +243,7 @@ public void testProxyProduceConsume() throws Exception { public void testClientReconnectsToBrokerOnProxyClosing() throws Exception { var namespaceName = NamespaceName.get("public", "default"); var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, - BrokerTestUtil.newUniqueName("testProxyReconnect")); + BrokerTestUtil.newUniqueName("testClientReconnectsToBrokerOnProxyClosing")); @Cleanup("shutdownNow") var threadPool = Executors.newCachedThreadPool(); From a1e48109b28d57ba2458402bfbadc45c229dadad Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 22 Dec 2023 14:12:00 -0800 Subject: [PATCH 8/8] Revert invocationCount --- .../proxy/server/ProxyWithExtensibleLoadManagerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index b0668f5ea07d2..3a787a8b35995 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -162,7 +162,7 @@ public void proxyCleanup() throws Exception { } } - @Test(timeOut = TEST_TIMEOUT_MS, invocationCount = 100) + @Test(timeOut = TEST_TIMEOUT_MS) public void testProxyProduceConsume() throws Exception { var namespaceName = NamespaceName.get("public", "default"); var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, @@ -239,7 +239,7 @@ public void testProxyProduceConsume() throws Exception { verify(consumerLookupServiceSpy, never()).getBroker(topicName); } - @Test(timeOut = TEST_TIMEOUT_MS, invocationCount = 100) + @Test(timeOut = TEST_TIMEOUT_MS) public void testClientReconnectsToBrokerOnProxyClosing() throws Exception { var namespaceName = NamespaceName.get("public", "default"); var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName,