From da71ec2b2a6682ffbab60843756f638bedbd4460 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sat, 5 Jun 2021 21:37:53 -0700 Subject: [PATCH] Make client-admin-api to use interfaces with builders instead of POJOs (#10818) ### Motivation Instead of using POJOs types in the Java client-admin API, use interfaces ### Modifications * Converted more POJOs into interfaces * Added builders to construct instances of the interfaces without directly using the implementation classes Note: for easier reviewing, there are 2 commits in this PR: * https://github.com/apache/pulsar/commit/f14ce733f69633143bd3bf4b4f3ae99a2ea93551 includes the changes to production code * https://github.com/apache/pulsar/commit/23b6f74c608cefc5da14ce1d2b29f34efde6870f changes to the unit tests --- .../client/TlsProducerConsumerBase.java | 9 +- .../authorization/AuthorizationProvider.java | 3 +- .../authorization/AuthorizationService.java | 3 +- .../PulsarAuthorizationProvider.java | 25 +- .../broker/resources/ClusterResources.java | 5 +- .../broker/resources/TenantResources.java | 5 +- .../pulsar/PulsarClusterMetadataSetup.java | 13 +- .../org/apache/pulsar/PulsarStandalone.java | 24 +- .../apache/pulsar/broker/PulsarService.java | 8 +- .../pulsar/broker/admin/AdminResource.java | 38 +- .../pulsar/broker/admin/impl/BrokersBase.java | 4 +- .../broker/admin/impl/ClustersBase.java | 69 +-- .../broker/admin/impl/NamespacesBase.java | 40 +- .../admin/impl/PersistentTopicsBase.java | 73 ++-- .../pulsar/broker/admin/impl/TenantsBase.java | 5 +- .../pulsar/broker/admin/v1/Namespaces.java | 9 +- .../pulsar/broker/admin/v2/Bookies.java | 2 +- .../pulsar/broker/admin/v2/Namespaces.java | 10 +- .../broker/admin/v2/PersistentTopics.java | 12 +- .../resourcegroup/ResourceGroupService.java | 8 +- .../broker/service/BacklogQuotaManager.java | 21 +- .../pulsar/broker/service/BrokerService.java | 18 +- .../pulsar/broker/service/Consumer.java | 13 +- .../pulsar/broker/service/Producer.java | 12 +- .../pulsar/broker/service/Replicator.java | 4 +- .../pulsar/broker/service/ServerCnx.java | 4 +- .../pulsar/broker/service/StreamingStats.java | 8 +- .../apache/pulsar/broker/service/Topic.java | 4 +- .../NonPersistentReplicator.java | 6 +- .../NonPersistentSubscription.java | 12 +- .../nonpersistent/NonPersistentTopic.java | 33 +- .../persistent/DispatchRateLimiter.java | 38 +- .../persistent/PersistentReplicator.java | 6 +- .../persistent/PersistentSubscription.java | 13 +- .../service/persistent/PersistentTopic.java | 36 +- .../prometheus/NamespaceStatsAggregator.java | 12 +- .../pulsar/broker/web/PulsarWebResource.java | 6 +- .../common/naming/NamespaceBundleFactory.java | 2 +- .../common/naming/NamespaceBundles.java | 7 +- .../apache/pulsar/broker/ConfigHelper.java | 38 +- .../pulsar/broker/SLAMonitoringTest.java | 17 +- .../broker/admin/AdminApiDelayedDelivery.java | 30 +- .../admin/AdminApiGetLastMessageIdTest.java | 3 +- .../admin/AdminApiMaxUnackedMessages.java | 3 +- .../broker/admin/AdminApiOffloadTest.java | 10 +- .../admin/AdminApiSchemaAutoUpdateTest.java | 3 +- .../broker/admin/AdminApiSchemaTest.java | 3 +- .../AdminApiSchemaValidationEnforced.java | 3 +- .../pulsar/broker/admin/AdminApiTest.java | 402 +++++++++--------- .../pulsar/broker/admin/AdminApiTest2.java | 276 ++++++------ .../broker/admin/AdminApiTlsAuthTest.java | 3 +- .../apache/pulsar/broker/admin/AdminTest.java | 114 +++-- .../pulsar/broker/admin/BookiesApiTest.java | 16 +- .../admin/BrokerAdminClientTlsAuthTest.java | 5 +- .../broker/admin/CreateSubscriptionTest.java | 8 +- .../broker/admin/IncrementPartitionsTest.java | 3 +- .../pulsar/broker/admin/NamespacesTest.java | 103 +++-- .../broker/admin/PersistentTopicsTest.java | 11 +- .../broker/admin/ResourceGroupsTest.java | 3 +- .../broker/admin/TopicMessageTTLTest.java | 5 +- .../admin/TopicPoliciesDisableTest.java | 17 +- .../broker/admin/TopicPoliciesTest.java | 224 +++++++--- .../broker/admin/v1/V1_AdminApiTest.java | 287 +++++++------ .../broker/admin/v1/V1_AdminApiTest2.java | 86 ++-- .../admin/v3/AdminApiTransactionTest.java | 3 +- .../pulsar/broker/auth/AuthLogsTest.java | 3 +- .../pulsar/broker/auth/AuthorizationTest.java | 3 +- .../auth/MockAuthorizationProvider.java | 4 +- .../auth/MockedPulsarServiceBaseTest.java | 3 +- .../AntiAffinityNamespaceGroupTest.java | 15 +- .../LeaderElectionServiceTest.java | 3 +- .../broker/loadbalance/LoadBalancerTest.java | 84 ++-- .../ModularLoadManagerImplTest.java | 3 +- .../SimpleLoadManagerImplTest.java | 28 +- .../lookup/http/HttpTopicLookupv2Test.java | 7 +- .../namespace/NamespaceCreateBundlesTest.java | 9 +- .../OwnerShipCacheForCurrentServerTest.java | 3 +- .../ResourceGroupServiceTest.java | 3 +- .../ResourceGroupUsageAggregationTest.java | 10 +- .../ResourceUsageTransportManagerTest.java | 3 +- .../service/BacklogQuotaManagerTest.java | 180 +++++--- .../broker/service/BkEnsemblesTestBase.java | 3 +- .../service/BrokerBookieIsolationTest.java | 104 +++-- ...erServiceAutoSubscriptionCreationTest.java | 8 +- .../BrokerServiceAutoTopicCreationTest.java | 37 +- .../broker/service/BrokerServiceTest.java | 114 ++--- .../pulsar/broker/service/BrokerTestBase.java | 3 +- .../service/EnableProxyProtocolTest.java | 12 +- .../broker/service/MaxMessageSizeTest.java | 3 +- .../broker/service/PeerReplicatorTest.java | 12 +- .../service/PersistentQueueE2ETest.java | 8 +- .../broker/service/PersistentTopicTest.java | 2 +- .../pulsar/broker/service/RackAwareTest.java | 5 +- .../service/ReplicatorRateLimiterTest.java | 48 ++- .../pulsar/broker/service/ReplicatorTest.java | 17 +- .../broker/service/ReplicatorTestBase.java | 30 +- ...temTopicBasedTopicPoliciesServiceTest.java | 3 +- .../pulsar/broker/service/TopicOwnerTest.java | 10 +- .../nonpersistent/NonPersistentTopicTest.java | 24 +- .../persistent/DelayedDeliveryTest.java | 21 +- .../PersistentSubscriptionTest.java | 6 +- .../persistent/PersistentTopicTest.java | 24 +- .../ReplicatedSubscriptionConfigTest.java | 14 +- .../broker/stats/ConsumerStatsTest.java | 25 +- .../broker/stats/SubscriptionStatsTest.java | 18 +- .../broker/stats/TransactionMetricsTest.java | 6 +- ...NamespaceEventsSystemTopicServiceTest.java | 3 +- .../TopicTransactionBufferRecoverTest.java | 3 +- .../TransactionClientReconnectTest.java | 3 +- .../transaction/TransactionConsumeTest.java | 3 +- .../transaction/TransactionProduceTest.java | 3 +- .../buffer/TransactionBufferClientTest.java | 3 +- .../buffer/TransactionLowWaterMarkTest.java | 3 +- .../buffer/TransactionStablePositionTest.java | 3 +- .../PendingAckInMemoryDeleteTest.java | 3 +- .../pendingack/PendingAckPersistentTest.java | 2 +- .../pulsar/broker/web/WebServiceTest.java | 18 +- .../ZooKeeperSessionExpireRecoveryTest.java | 7 +- .../AuthenticatedProducerConsumerTest.java | 32 +- ...enticationTlsHostnameVerificationTest.java | 5 +- .../AuthorizationProducerConsumerTest.java | 10 +- .../client/api/BrokerServiceLookupTest.java | 8 +- .../api/ClientDeduplicationFailureTest.java | 42 +- .../api/DispatcherBlockConsumerTest.java | 24 +- .../api/MessageDispatchThrottlingTest.java | 115 ++++- .../client/api/NonPersistentTopicTest.java | 60 +-- .../client/api/ProducerConsumerBase.java | 3 +- ...ListenersWithInternalListenerNameTest.java | 17 +- .../api/SimpleProducerConsumerStatTest.java | 2 +- .../api/SimpleProducerConsumerTest.java | 14 +- ...criptionMessageDispatchThrottlingTest.java | 48 ++- .../apache/pulsar/client/api/TenantTest.java | 5 +- .../client/api/TlsProducerConsumerBase.java | 10 +- ...okenAuthenticatedProducerConsumerTest.java | 3 +- ...uth2AuthenticatedProducerConsumerTest.java | 5 +- .../pulsar/client/api/TopicReaderTest.java | 12 +- .../api/v1/V1_ProducerConsumerBase.java | 3 +- .../impl/AdminApiKeyStoreTlsAuthTest.java | 9 +- .../impl/BrokerClientIntegrationTest.java | 3 +- .../impl/ConsumerConfigurationTest.java | 3 +- .../impl/DispatchAccordingPermitsTests.java | 2 +- ...reTlsProducerConsumerTestWithAuthTest.java | 9 +- ...lsProducerConsumerTestWithoutAuthTest.java | 9 +- .../client/impl/MessageChunkingTest.java | 2 +- .../pulsar/client/impl/MessageParserTest.java | 3 +- .../client/impl/MultiTopicsReaderTest.java | 3 +- .../pulsar/client/impl/RawReaderTest.java | 3 +- .../apache/pulsar/client/impl/ReaderTest.java | 9 +- .../pulsar/client/impl/SchemaDeleteTest.java | 3 +- .../client/impl/TopicsConsumerImplTest.java | 17 +- .../client/impl/TransactionEndToEndTest.java | 3 +- .../pulsar/compaction/CompactedTopicTest.java | 3 +- .../pulsar/compaction/CompactionTest.java | 3 +- .../pulsar/compaction/CompactorTest.java | 3 +- .../service/web/DiscoveryServiceWebTest.java | 3 +- .../worker/PulsarFunctionE2ESecurityTest.java | 40 +- .../worker/PulsarFunctionLocalRunTest.java | 97 +++-- .../worker/PulsarFunctionPublishTest.java | 45 +- .../worker/PulsarFunctionTlsTest.java | 7 +- .../worker/PulsarWorkerAssignmentTest.java | 26 +- .../pulsar/io/AbstractPulsarE2ETest.java | 12 +- .../pulsar/io/PulsarBatchSourceE2ETest.java | 22 +- .../pulsar/io/PulsarFunctionAdminTest.java | 10 +- .../pulsar/io/PulsarFunctionE2ETest.java | 111 ++--- .../apache/pulsar/io/PulsarSinkE2ETest.java | 28 +- .../apache/pulsar/io/PulsarSourceE2ETest.java | 22 +- .../schema/PartitionedTopicSchemaTest.java | 3 +- .../org/apache/pulsar/schema/SchemaTest.java | 9 +- .../SchemaCompatibilityCheckTest.java | 9 +- .../SchemaTypeCompatibilityCheckTest.java | 9 +- .../proxy/ProxyAuthorizationTest.java | 3 +- .../proxy/ProxyPublishConsumeTest.java | 10 +- pulsar-client-admin-api/pom.xml | 3 - .../client/admin/utils/ReflectionUtils.java | 9 +- .../common/policies/data/AuthPolicies.java | 41 +- .../policies/data/AutoFailoverPolicyData.java | 17 +- .../AutoSubscriptionCreationOverride.java | 90 ++-- .../data/AutoTopicCreationOverride.java | 65 +-- .../common/policies/data/BacklogQuota.java | 68 +-- .../data/BookieAffinityGroupData.java | 36 +- .../common/policies/data/BookieInfo.java | 23 +- .../policies/data/BookiesClusterInfo.java | 21 +- .../common/policies/data/BrokerInfo.java | 20 +- .../data/BrokerNamespaceIsolationData.java | 20 +- .../common/policies/data/BrokerStatus.java | 49 +-- .../common/policies/data/BundlesData.java | 58 +-- .../common/policies/data/ClusterData.java | 71 ++-- .../common/policies/data/ConsumerStats.java | 119 +----- .../data/DelayedDeliveryPolicies.java | 27 +- .../common/policies/data/DispatchRate.java | 55 +-- .../common/policies/data/FailureDomain.java | 10 +- .../policies/data/FunctionInstanceStats.java | 4 - .../data/FunctionInstanceStatsData.java | 10 +- .../data/FunctionInstanceStatsDataBase.java | 10 - .../common/policies/data/FunctionStats.java | 20 +- .../policies/data/NamespaceIsolationData.java | 20 +- .../data/NonPersistentPublisherStats.java | 13 +- .../data/NonPersistentReplicatorStats.java | 13 +- .../data/NonPersistentSubscriptionStats.java | 20 +- .../data/NonPersistentTopicStats.java | 46 +- .../common/policies/data/OffloadPolicies.java | 67 +-- .../policies/data/PartitionedTopicStats.java | 26 +- .../pulsar/common/policies/data/Policies.java | 11 +- .../common/policies/data/PublisherStats.java | 106 +---- .../common/policies/data/ReplicatorStats.java | 41 +- .../policies/data/SubscriptionStats.java | 125 ++---- .../common/policies/data/TenantInfo.java | 13 +- .../common/policies/data/TopicStats.java | 145 +------ .../AutoSubscriptionCreationOverrideImpl.java | 53 +++ .../impl/AutoTopicCreationOverrideImpl.java | 90 ++++ .../policies/data/impl/BacklogQuotaImpl.java | 63 +++ .../impl/BookieAffinityGroupDataImpl.java | 61 +++ .../policies/data/impl/BookieInfoImpl.java | 58 +++ .../data/impl/BookiesClusterInfoImpl.java | 54 +++ .../policies/data/impl/BrokerInfoImpl.java | 51 +++ .../policies/data/impl/BrokerStatusImpl.java | 74 ++++ .../policies/data/impl/BundlesDataImpl.java | 59 +++ .../impl/DelayedDeliveryPoliciesImpl.java | 58 +++ .../policies/data/impl/DispatchRateImpl.java | 76 ++++ .../policies/data/impl/package-info.java | 19 + .../impl/AutoFailoverPolicyFactory.java | 4 +- .../policies/impl/MinAvailablePolicy.java | 31 +- .../client/admin/internal/NamespacesImpl.java | 4 +- .../client/admin/internal/TenantsImpl.java | 2 +- .../client/admin/internal/TopicsImpl.java | 2 +- .../pulsar/admin/cli/PulsarAdminToolTest.java | 152 +++++-- pulsar-client-tools/pom.xml | 5 + .../apache/pulsar/admin/cli/CmdBookies.java | 6 +- .../apache/pulsar/admin/cli/CmdClusters.java | 77 ++-- .../cli/CmdNamespaceIsolationPolicy.java | 27 +- .../pulsar/admin/cli/CmdNamespaces.java | 54 ++- .../apache/pulsar/admin/cli/CmdTopics.java | 32 +- .../pulsar/admin/cli/utils/CmdUtils.java | 4 +- .../pulsar/admin/cli/TestCmdClusters.java | 7 +- pulsar-common/pom.xml | 7 + .../admin/internal/data/AuthPoliciesImpl.java | 78 ++++ .../admin/internal/data/package-info.java | 19 + .../data/AutoFailoverPolicyDataImpl.java | 50 ++- .../BrokerNamespaceIsolationDataImpl.java | 58 ++- .../common/policies/data/ClusterDataImpl.java | 205 +++++---- .../policies/data/FailureDomainImpl.java | 40 +- .../data/FunctionInstanceStatsDataImpl.java | 2 +- .../policies/data/FunctionStatsImpl.java | 7 +- .../data/NamespaceIsolationDataImpl.java | 88 ++-- .../policies/data/OffloadPoliciesImpl.java | 325 +++++++------- .../policies/data/PersistentTopicStats.java | 2 +- .../common/policies/data/PoliciesUtil.java | 12 +- .../common/policies/data/TenantInfoImpl.java | 53 ++- .../common/policies/data/TopicPolicies.java | 10 +- .../data/stats/ConsumerStatsImpl.java | 156 +++++++ .../NonPersistentPublisherStatsImpl.java | 44 ++ .../NonPersistentReplicatorStatsImpl.java | 44 ++ .../NonPersistentSubscriptionStatsImpl.java | 52 +++ .../stats/NonPersistentTopicStatsImpl.java | 102 +++++ .../data/stats/PartitionedTopicStatsImpl.java | 59 +++ .../data/stats/PublisherStatsImpl.java | 145 +++++++ .../data/stats/ReplicatorStatsImpl.java | 79 ++++ .../data/stats/SubscriptionStatsImpl.java | 173 ++++++++ .../policies/data/stats/TopicStatsImpl.java | 213 ++++++++++ .../impl/NamespaceIsolationPolicies.java | 15 +- .../impl/NamespaceIsolationPolicyImpl.java | 11 +- .../common/util/ObjectMapperFactory.java | 67 ++- .../data/AutoFailoverPolicyDataTest.java | 30 +- .../data/AutoTopicCreationOverrideTest.java | 46 +- .../policies/data/BacklogQuotaTest.java | 50 ++- .../common/policies/data/ClusterDataTest.java | 87 +++- .../policies/data/ConsumerStatsTest.java | 3 +- .../policies/data/LocalPolicesTest.java | 32 +- .../data/NamespaceIsolationDataTest.java | 77 +++- .../common/policies/data/OldPolicies.java | 2 +- .../data/PartitionedTopicStatsTest.java | 12 +- .../data/PersistentTopicStatsTest.java | 32 +- .../policies/data/PoliciesDataTest.java | 25 +- .../policies/data/PublisherStatsTest.java | 9 +- .../policies/data/ReplicatorStatsTest.java | 5 +- .../impl/AutoFailoverPolicyFactoryTest.java | 18 +- .../impl/NamespaceIsolationPoliciesTest.java | 58 ++- .../NamespaceIsolationPolicyImplTest.java | 92 ++-- .../common/util/ObjectMapperFactoryTest.java | 9 +- .../service/BrokerDiscoveryProvider.java | 3 +- .../service/web/DiscoveryServiceWebTest.java | 4 +- .../functions/worker/FunctionActioner.java | 6 +- .../functions/worker/MembershipManager.java | 16 +- .../functions/worker/PulsarWorkerService.java | 10 +- .../worker/rest/api/FunctionsImplTest.java | 14 +- .../server/AuthedAdminProxyHandlerTest.java | 3 +- ...roxyAuthenticatedProducerConsumerTest.java | 9 +- .../ProxyEnableHAProxyProtocolTest.java | 12 +- .../server/ProxyWithAuthorizationNegTest.java | 3 +- .../server/ProxyWithAuthorizationTest.java | 9 +- .../server/ProxyWithJwtAuthorizationTest.java | 7 +- .../ProxyWithoutServiceDiscoveryTest.java | 3 +- .../SuperUserAuthedAdminProxyHandlerTest.java | 5 +- .../sql/presto/TestCacheSizeAllocator.java | 3 +- .../testclient/PerformanceProducerTest.java | 3 +- .../pulsar/websocket/WebSocketService.java | 27 +- .../ZkBookieRackAffinityMappingTest.java | 10 +- ...atedBookieEnsemblePlacementPolicyTest.java | 70 +-- .../functions/PulsarFunctionsTest.java | 14 +- .../tests/integration/proxy/TestProxy.java | 2 +- .../SimpleProducerConsumerTest.java | 7 +- .../SimpleProducerConsumerTest.java | 5 +- 302 files changed, 6239 insertions(+), 3563 deletions(-) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoSubscriptionCreationOverrideImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoTopicCreationOverrideImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BacklogQuotaImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieAffinityGroupDataImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieInfoImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookiesClusterInfoImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerInfoImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerStatusImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BundlesDataImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DelayedDeliveryPoliciesImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DispatchRateImpl.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/package-info.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/AuthPoliciesImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/package-info.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentSubscriptionStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PartitionedTopicStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java diff --git a/bouncy-castle/bcfips-include-test/src/test/java/org/apache/pulsar/client/TlsProducerConsumerBase.java b/bouncy-castle/bcfips-include-test/src/test/java/org/apache/pulsar/client/TlsProducerConsumerBase.java index c73d4a04da418..a9c4f35976fc3 100644 --- a/bouncy-castle/bcfips-include-test/src/test/java/org/apache/pulsar/client/TlsProducerConsumerBase.java +++ b/bouncy-castle/bcfips-include-test/src/test/java/org/apache/pulsar/client/TlsProducerConsumerBase.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.annotations.AfterMethod; @@ -104,8 +105,12 @@ protected void internalSetUpForNamespace() throws Exception { admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrlTls.toString()) .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(false) .authentication(AuthenticationTls.class.getName(), authParams).build()); - admin.clusters().createCluster(clusterName, new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster(clusterName, ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/my-ns"); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index cfb191b8fc1f4..443a2a8594d1e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -33,6 +33,7 @@ import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TenantOperation; @@ -78,7 +79,7 @@ default CompletableFuture isSuperUser(String role, ServiceConfiguration * @return a CompletableFuture containing a boolean in which true means the role is an admin user * and false if it is not */ - default CompletableFuture isTenantAdmin(String tenant, String role, TenantInfoImpl tenantInfo, + default CompletableFuture isTenantAdmin(String tenant, String role, TenantInfo tenantInfo, AuthenticationDataSource authenticationData) { return CompletableFuture.completedFuture(role != null && tenantInfo.getAdminRoles() != null && tenantInfo.getAdminRoles().contains(role) ? true : false); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index 83a9944633643..4c4963a68bab1 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -29,6 +29,7 @@ import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.policies.data.TopicOperation; @@ -80,7 +81,7 @@ public CompletableFuture isSuperUser(String user, AuthenticationDataSou return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured")); } - public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfoImpl tenantInfo, + public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfo tenantInfo, AuthenticationDataSource authenticationData) { if (provider != null) { return provider.isTenantAdmin(tenant, role, tenantInfo, authenticationData); 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 ecd17f8a7f7b3..d5029713918db 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 @@ -39,6 +39,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.policies.data.TopicOperation; @@ -118,7 +119,8 @@ public CompletableFuture canConsumeAsync(TopicName topicName, String ro if (isNotBlank(subscription)) { // validate if role is authorize to access subscription. (skip validatation if authorization // list is empty) - Set roles = policies.get().auth_policies.subscription_auth_roles.get(subscription); + Set roles = policies.get().auth_policies + .getSubscriptionAuthentication().get(subscription); if (roles != null && !roles.isEmpty() && !roles.contains(role)) { log.warn("[{}] is not authorized to subscribe on {}-{}", role, topicName, subscription); PulsarServerException ex = new PulsarServerException( @@ -242,7 +244,8 @@ private CompletableFuture allowTheSpecifiedActionOpsAsync(NamespaceName log.debug("Policies node couldn't be found for namespace : {}", namespaceName); } } else { - Map> namespaceRoles = policies.get().auth_policies.namespace_auth; + Map> namespaceRoles = policies.get() + .auth_policies.getNamespaceAuthentication(); Set namespaceActions = namespaceRoles.get(role); if (namespaceActions != null && namespaceActions.contains(authAction)) { // The role has namespace level permission @@ -294,7 +297,7 @@ public CompletableFuture grantPermissionAsync(NamespaceName namespaceName, final String policiesPath = String.format("/%s/%s/%s", "admin", POLICIES, namespaceName.toString()); try { pulsarResources.getNamespaceResources().set(policiesPath, (policies)->{ - policies.auth_policies.namespace_auth.put(role, actions); + policies.auth_policies.getNamespaceAuthentication().put(role, actions); return policies; }); log.info("[{}] Successfully granted access for role {}: {} - namespace {}", role, role, actions, @@ -344,15 +347,15 @@ private CompletableFuture updateSubscriptionPermissionAsync(NamespaceName Policies policies = pulsarResources.getNamespaceResources().get(policiesPath) .orElseThrow(() -> new NotFoundException(policiesPath + " not found")); if (remove) { - if (policies.auth_policies.subscription_auth_roles.get(subscriptionName) != null) { - policies.auth_policies.subscription_auth_roles.get(subscriptionName).removeAll(roles); + if (policies.auth_policies.getSubscriptionAuthentication().get(subscriptionName) != null) { + policies.auth_policies.getSubscriptionAuthentication().get(subscriptionName).removeAll(roles); }else { log.info("[{}] Couldn't find role {} while revoking for sub = {}", namespace, subscriptionName, roles); result.completeExceptionally(new IllegalArgumentException("couldn't find subscription")); return result; } } else { - policies.auth_policies.subscription_auth_roles.put(subscriptionName, roles); + policies.auth_policies.getSubscriptionAuthentication().put(subscriptionName, roles); } pulsarResources.getNamespaceResources().set(policiesPath, (data)->policies); @@ -400,7 +403,8 @@ public CompletableFuture checkPermission(TopicName topicName, String ro log.debug("Policies node couldn't be found for topic : {}", topicName); } } else { - Map> namespaceRoles = policies.get().auth_policies.namespace_auth; + Map> namespaceRoles = policies.get().auth_policies + .getNamespaceAuthentication(); Set namespaceActions = namespaceRoles.get(role); if (namespaceActions != null && namespaceActions.contains(action)) { // The role has namespace level permission @@ -408,7 +412,7 @@ public CompletableFuture checkPermission(TopicName topicName, String ro return; } - Map> topicRoles = policies.get().auth_policies.destination_auth + Map> topicRoles = policies.get().auth_policies.getTopicAuthentication() .get(topicName.toString()); if (topicRoles != null && role != null) { // Topic has custom policy @@ -440,7 +444,8 @@ public CompletableFuture checkPermission(TopicName topicName, String ro // We can also check the permission of partitioned topic. // For https://github.com/apache/pulsar/issues/10300 if (topicName.isPartitioned()) { - topicRoles = policies.get().auth_policies.destination_auth.get(topicName.getPartitionedTopicName()); + topicRoles = policies.get().auth_policies + .getTopicAuthentication().get(topicName.getPartitionedTopicName()); if (topicRoles != null) { // Topic has custom policy Set topicActions = topicRoles.get(role); @@ -634,7 +639,7 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, return CompletableFuture.completedFuture(true); } else { try { - TenantInfoImpl tenantInfo = pulsarResources.getTenantResources() + TenantInfo tenantInfo = pulsarResources.getTenantResources() .get(path(POLICIES, tenantName)) .orElseThrow(() -> new RestException(Response.Status.NOT_FOUND, "Tenant does not exist")); return isTenantAdmin(tenantName, role, tenantInfo, authData); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java index dcfc52dd7c074..a92ecf16f76d7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java @@ -22,19 +22,20 @@ import java.util.Set; import lombok.Getter; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; -public class ClusterResources extends BaseResources { +public class ClusterResources extends BaseResources { public static final String CLUSTERS_ROOT = "/admin/clusters"; @Getter private FailureDomainResources failureDomainResources; public ClusterResources(MetadataStoreExtended store, int operationTimeoutSec) { - super(store, ClusterDataImpl.class, operationTimeoutSec); + super(store, ClusterData.class, operationTimeoutSec); this.failureDomainResources = new FailureDomainResources(store, FailureDomainImpl.class, operationTimeoutSec); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java index 3ae40ede2f75f..127332e1332ea 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java @@ -18,11 +18,12 @@ */ package org.apache.pulsar.broker.resources; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; -public class TenantResources extends BaseResources { +public class TenantResources extends BaseResources { public TenantResources(MetadataStoreExtended store, int operationTimeoutSec) { - super(store, TenantInfoImpl.class, operationTimeoutSec); + super(store, TenantInfo.class, operationTimeoutSec); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index 6eb415394ab2a..c7b6711192d45 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -36,7 +36,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; +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.pulsar.common.util.ObjectMapperFactory; @@ -236,15 +236,18 @@ public static void main(String[] args) throws Exception { createMetadataNode(configStore, "/admin/clusters", new byte[0]); - ClusterDataImpl clusterData = new ClusterDataImpl(arguments.clusterWebServiceUrl, - arguments.clusterWebServiceUrlTls, arguments.clusterBrokerServiceUrl, - arguments.clusterBrokerServiceUrlTls); + ClusterData clusterData = ClusterData.builder() + .serviceUrl(arguments.clusterWebServiceUrl) + .serviceUrlTls(arguments.clusterWebServiceUrlTls) + .brokerServiceUrl(arguments.clusterBrokerServiceUrl) + .brokerServiceUrlTls(arguments.clusterBrokerServiceUrlTls) + .build(); byte[] clusterDataJson = ObjectMapperFactory.getThreadLocal().writeValueAsBytes(clusterData); createMetadataNode(configStore, "/admin/clusters/" + arguments.cluster, clusterDataJson); // Create marker for "global" cluster - ClusterDataImpl globalClusterData = new ClusterDataImpl(null, null); + ClusterData globalClusterData = ClusterData.builder().build(); byte[] globalClusterDataJson = ObjectMapperFactory.getThreadLocal().writeValueAsBytes(globalClusterData); createMetadataNode(configStore, "/admin/clusters/global", globalClusterDataJson); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index dfaafd2366001..b9d49636ee819 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -34,7 +34,8 @@ import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; @@ -305,8 +306,10 @@ public void start() throws Exception { webServiceUrl.toString()).authentication( config.getBrokerClientAuthenticationPlugin(), config.getBrokerClientAuthenticationParameters()).build(); - ClusterDataImpl clusterData = - new ClusterDataImpl(webServiceUrl.toString(), null, brokerServiceUrl, null); + ClusterData clusterData = ClusterData.builder() + .serviceUrl(webServiceUrl.toString()) + .brokerServiceUrl(brokerServiceUrl) + .build(); createSampleNameSpace(clusterData, cluster); } else { URL webServiceUrlTls = new URL( @@ -333,8 +336,10 @@ public void start() throws Exception { } admin = builder.build(); - ClusterDataImpl clusterData = new ClusterDataImpl(null, webServiceUrlTls.toString(), - null, brokerServiceUrlTls); + ClusterData clusterData = ClusterData.builder() + .serviceUrlTls(webServiceUrlTls.toString()) + .brokerServiceUrlTls(brokerServiceUrlTls) + .build(); createSampleNameSpace(clusterData, cluster); } @@ -355,7 +360,10 @@ private void createNameSpace(String cluster, String publicTenant, String default try { if (!admin.tenants().getTenants().contains(publicTenant)) { admin.tenants().createTenant(publicTenant, - new TenantInfoImpl(Sets.newHashSet(config.getSuperUserRoles()), Sets.newHashSet(cluster))); + TenantInfo.builder() + .adminRoles(Sets.newHashSet(config.getSuperUserRoles())) + .allowedClusters(Sets.newHashSet(cluster)) + .build()); } if (!admin.namespaces().getNamespaces(publicTenant).contains(defaultNamespace)) { admin.namespaces().createNamespace(defaultNamespace); @@ -367,7 +375,7 @@ private void createNameSpace(String cluster, String publicTenant, String default } } - private void createSampleNameSpace(ClusterDataImpl clusterData, String cluster) { + private void createSampleNameSpace(ClusterData clusterData, String cluster) { // Create a sample namespace final String tenant = "sample"; final String globalCluster = "global"; @@ -383,7 +391,7 @@ private void createSampleNameSpace(ClusterDataImpl clusterData, String cluster) try { admin.clusters().getCluster(globalCluster); } catch (PulsarAdminException.NotFoundException ex) { - admin.clusters().createCluster(globalCluster, new ClusterDataImpl(null, null)); + admin.clusters().createCluster(globalCluster, ClusterData.builder().build()); } if (!admin.tenants().getTenants().contains(tenant)) { 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 7b71a9f7803a1..be85f545e62ba 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 @@ -729,8 +729,12 @@ public Boolean get() { this.brokerServiceUrlTls = brokerUrlTls(config); if (null != this.webSocketService) { - ClusterDataImpl clusterData = - new ClusterDataImpl(webServiceAddress, webServiceAddressTls, brokerServiceUrl, brokerServiceUrlTls); + ClusterDataImpl clusterData = ClusterDataImpl.builder() + .serviceUrl(webServiceAddress) + .serviceUrlTls(webServiceAddressTls) + .brokerServiceUrl(brokerServiceUrl) + .brokerServiceUrlTls(brokerServiceUrlTls) + .build(); this.webSocketService.setLocalCluster(clusterData); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 473e0526eff1d..88064971e7de9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -51,7 +51,6 @@ import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BundlesData; -import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; @@ -59,6 +58,7 @@ import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.TopicPolicies; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -401,28 +401,28 @@ protected void checkTopicLevelPolicyEnable() { } } - protected DispatchRate dispatchRate() { - return new DispatchRate( - pulsar().getConfiguration().getDispatchThrottlingRatePerTopicInMsg(), - pulsar().getConfiguration().getDispatchThrottlingRatePerTopicInByte(), - 1 - ); + protected DispatchRateImpl dispatchRate() { + return DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(config().getDispatchThrottlingRatePerTopicInMsg()) + .dispatchThrottlingRateInByte(config().getDispatchThrottlingRatePerTopicInByte()) + .ratePeriodInSecond(1) + .build(); } - protected DispatchRate subscriptionDispatchRate() { - return new DispatchRate( - pulsar().getConfiguration().getDispatchThrottlingRatePerSubscriptionInMsg(), - pulsar().getConfiguration().getDispatchThrottlingRatePerSubscriptionInByte(), - 1 - ); + protected DispatchRateImpl subscriptionDispatchRate() { + return DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(config().getDispatchThrottlingRatePerSubscriptionInMsg()) + .dispatchThrottlingRateInByte(config().getDispatchThrottlingRatePerSubscriptionInByte()) + .ratePeriodInSecond(1) + .build(); } - protected DispatchRate replicatorDispatchRate() { - return new DispatchRate( - pulsar().getConfiguration().getDispatchThrottlingRatePerReplicatorInMsg(), - pulsar().getConfiguration().getDispatchThrottlingRatePerReplicatorInByte(), - 1 - ); + protected DispatchRateImpl replicatorDispatchRate() { + return DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(config().getDispatchThrottlingRatePerReplicatorInMsg()) + .dispatchThrottlingRateInByte(config().getDispatchThrottlingRatePerReplicatorInByte()) + .ratePeriodInSecond(1) + .build(); } protected SubscribeRate subscribeRate() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 8c80e38e8f620..2e80dd3391ab5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -110,9 +110,7 @@ public BrokerInfo getLeaderBroker() throws Exception { try { LeaderBroker leaderBroker = pulsar().getLeaderElectionService().getCurrentLeader() .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Couldn't find leader broker")); - BrokerInfo brokerInfo = new BrokerInfo(); - brokerInfo.setServiceUrl(leaderBroker.getServiceUrl()); - return brokerInfo; + return BrokerInfo.builder().serviceUrl(leaderBroker.getServiceUrl()).build(); } catch (Exception e) { LOG.error("[{}] Failed to get the information of the leader broker.", clientAppId(), e); throw new RestException(e); 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 dd1f3faee8a60..76cd60209e689 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 @@ -28,6 +28,7 @@ import io.swagger.annotations.ApiResponses; import io.swagger.annotations.Example; import io.swagger.annotations.ExampleProperty; +import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; @@ -56,9 +57,12 @@ import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.common.naming.Constants; import org.apache.pulsar.common.naming.NamedEntity; +import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationData; 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.FailureDomainImpl; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicyImpl; @@ -104,7 +108,7 @@ public Set getClusters() throws Exception { @ApiResponse(code = 404, message = "Cluster doesn't exist."), @ApiResponse(code = 500, message = "Internal server error.") }) - public ClusterDataImpl getCluster( + public ClusterData getCluster( @ApiParam( value = "The cluster name", required = true @@ -218,10 +222,7 @@ public void updateCluster( validatePoliciesReadOnlyAccess(); try { - clusterResources().set(path("clusters", cluster), old -> { - old.update(clusterData); - return old; - }); + clusterResources().set(path("clusters", cluster), old -> clusterData); log.info("[{}] Updated cluster {}", clientAppId(), cluster); } catch (NotFoundException e) { log.warn("[{}] Failed to update cluster {}: Does not exist", clientAppId(), cluster); @@ -293,10 +294,11 @@ public void setPeerClusterNames( } try { - clusterResources().set(path("clusters", cluster), old -> { - old.setPeerClusterNames(peerClusterNames); - return old; - }); + clusterResources().set(path("clusters", cluster), old -> + old.clone() + .peerClusterNames(peerClusterNames) + .build() + ); log.info("[{}] Successfully added peer-cluster {} for {}", clientAppId(), peerClusterNames, cluster); } catch (NotFoundException e) { log.warn("[{}] Failed to update cluster {}: Does not exist", clientAppId(), cluster); @@ -329,7 +331,7 @@ public Set getPeerCluster( ) { validateSuperUserAccess(); try { - ClusterDataImpl clusterData = clusterResources().get(path("clusters", cluster)) + ClusterData clusterData = clusterResources().get(path("clusters", cluster)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster does not exist")); return clusterData.getPeerClusterNames(); } catch (Exception e) { @@ -442,7 +444,7 @@ private void deleteFailureDomain(String clusterPath) { @ApiResponse(code = 404, message = "Cluster doesn't exist."), @ApiResponse(code = 500, message = "Internal server error.") }) - public Map getNamespaceIsolationPolicies( + public Map getNamespaceIsolationPolicies( @ApiParam( value = "The cluster name", required = true @@ -480,7 +482,7 @@ public Map getNamespaceIsolationPolicies( @ApiResponse(code = 412, message = "Cluster doesn't exist."), @ApiResponse(code = 500, message = "Internal server error.") }) - public NamespaceIsolationDataImpl getNamespaceIsolationPolicy( + public NamespaceIsolationData getNamespaceIsolationPolicy( @ApiParam( value = "The cluster name", required = true @@ -530,7 +532,7 @@ public NamespaceIsolationDataImpl getNamespaceIsolationPolicy( @ApiResponse(code = 412, message = "Cluster doesn't exist."), @ApiResponse(code = 500, message = "Internal server error.") }) - public List getBrokersWithNamespaceIsolationPolicy( + public List getBrokersWithNamespaceIsolationPolicy( @ApiParam( value = "The cluster name", required = true @@ -541,7 +543,7 @@ public List getBrokersWithNamespaceIsolationPo Set availableBrokers; final String nsIsolationPoliciesPath = AdminResource.path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); - Map nsPolicies; + Map nsPolicies; try { availableBrokers = pulsar().getLoadManager().get().getAvailableBrokers(); } catch (Exception e) { @@ -560,23 +562,23 @@ public List getBrokersWithNamespaceIsolationPo throw new RestException(e); } return availableBrokers.stream().map(broker -> { - BrokerNamespaceIsolationDataImpl brokerIsolationData = new BrokerNamespaceIsolationDataImpl(); - brokerIsolationData.brokerName = broker; + BrokerNamespaceIsolationData.Builder brokerIsolationData = BrokerNamespaceIsolationData.builder() + .brokerName(broker); if (nsPolicies != null) { + List namespaceRegexes = new ArrayList<>(); nsPolicies.forEach((name, policyData) -> { NamespaceIsolationPolicyImpl nsPolicyImpl = new NamespaceIsolationPolicyImpl(policyData); if (nsPolicyImpl.isPrimaryBroker(broker) || nsPolicyImpl.isSecondaryBroker(broker)) { - if (brokerIsolationData.namespaceRegex == null) { - brokerIsolationData.namespaceRegex = Lists.newArrayList(); - } - brokerIsolationData.namespaceRegex.addAll(policyData.namespaces); + namespaceRegexes.addAll(policyData.getNamespaces()); if (nsPolicyImpl.isPrimaryBroker(broker)) { - brokerIsolationData.isPrimary = true; + brokerIsolationData.primary(true); } } }); + + brokerIsolationData.namespaceRegex(namespaceRegexes); } - return brokerIsolationData; + return brokerIsolationData.build(); }).collect(Collectors.toList()); } @@ -593,7 +595,7 @@ public List getBrokersWithNamespaceIsolationPo @ApiResponse(code = 412, message = "Cluster doesn't exist."), @ApiResponse(code = 500, message = "Internal server error.") }) - public BrokerNamespaceIsolationDataImpl getBrokerWithNamespaceIsolationPolicy( + public BrokerNamespaceIsolationData getBrokerWithNamespaceIsolationPolicy( @ApiParam( value = "The cluster name", required = true @@ -609,7 +611,7 @@ public BrokerNamespaceIsolationDataImpl getBrokerWithNamespaceIsolationPolicy( validateClusterExists(cluster); final String nsIsolationPoliciesPath = AdminResource.path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); - Map nsPolicies; + Map nsPolicies; try { Optional nsPoliciesResult = namespaceIsolationPolicies() .getPolicies(nsIsolationPoliciesPath); @@ -621,23 +623,22 @@ public BrokerNamespaceIsolationDataImpl getBrokerWithNamespaceIsolationPolicy( log.error("[{}] Failed to get namespace isolation-policies {}", clientAppId(), cluster, e); throw new RestException(e); } - BrokerNamespaceIsolationDataImpl brokerIsolationData = new BrokerNamespaceIsolationDataImpl(); - brokerIsolationData.brokerName = broker; + BrokerNamespaceIsolationData.Builder brokerIsolationData = BrokerNamespaceIsolationData.builder() + .brokerName(broker); if (nsPolicies != null) { + List namespaceRegexes = new ArrayList<>(); nsPolicies.forEach((name, policyData) -> { NamespaceIsolationPolicyImpl nsPolicyImpl = new NamespaceIsolationPolicyImpl(policyData); boolean isPrimary = nsPolicyImpl.isPrimaryBroker(broker); if (isPrimary || nsPolicyImpl.isSecondaryBroker(broker)) { - if (brokerIsolationData.namespaceRegex == null) { - brokerIsolationData.namespaceRegex = Lists.newArrayList(); - } - brokerIsolationData.namespaceRegex.addAll(policyData.namespaces); - brokerIsolationData.isPrimary = isPrimary; - brokerIsolationData.policyName = name; + namespaceRegexes.addAll(policyData.getNamespaces()); + brokerIsolationData.primary(isPrimary); + brokerIsolationData.policyName(name); } }); + brokerIsolationData.namespaceRegex(namespaceRegexes); } - return brokerIsolationData; + return brokerIsolationData.build(); } @POST @@ -756,7 +757,7 @@ private void filterAndUnloadMatchedNameSpaces(AsyncResponse asyncResponse, int leftNssToHandle = nssNumber.decrementAndGet(); // if namespace match any policy regex, add it to ns list to be unload. - if (policyData.namespaces.stream() + if (policyData.getNamespaces().stream() .anyMatch(nsnameRegex -> namespaceName.matches(nsnameRegex))) { nssToUnload.add(namespaceName); } 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 6bae8f89b92ea..f6d69a0b96ed8 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 @@ -81,7 +81,7 @@ import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.BundlesData; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; @@ -99,6 +99,8 @@ import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; import org.apache.pulsar.common.policies.data.TenantOperation; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.apache.pulsar.metadata.api.MetadataStoreException.BadVersionException; @@ -192,7 +194,7 @@ protected void internalDeleteNamespace(AsyncResponse asyncResponse, boolean auth && !policies.replication_clusters.contains(config().getClusterName())) { // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); - ClusterDataImpl replClusterData = clusterResources().get( + ClusterData replClusterData = clusterResources().get( AdminResource.path("clusters", replCluster)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); @@ -356,7 +358,7 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo && !policies.replication_clusters.contains(config().getClusterName())) { // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); - ClusterDataImpl replClusterData = + ClusterData replClusterData = clusterResources().get(AdminResource.path("clusters", replCluster)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); @@ -509,7 +511,7 @@ protected void internalDeleteNamespaceBundle(String bundleRange, boolean authori && !policies.replication_clusters.contains(config().getClusterName())) { // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); - ClusterDataImpl replClusterData = + ClusterData replClusterData = clusterResources().get(AdminResource.path("clusters", replCluster)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); @@ -583,7 +585,7 @@ protected void internalDeleteNamespaceBundleForcefully(String bundleRange, boole && !policies.replication_clusters.contains(config().getClusterName())) { // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); - ClusterDataImpl replClusterData = + ClusterData replClusterData = clusterResources().get(AdminResource.path("clusters", replCluster)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); @@ -696,7 +698,7 @@ protected void internalRevokePermissionsOnNamespace(String role) { validatePoliciesReadOnlyAccess(); checkNotNull(role, "Role should not be null"); updatePolicies(path(POLICIES, namespaceName.toString()), policies ->{ - policies.auth_policies.namespace_auth.remove(role); + policies.auth_policies.getNamespaceAuthentication().remove(role); return policies; }); } @@ -797,11 +799,11 @@ protected void internalSetAutoTopicCreation(AsyncResponse asyncResponse, validateNamespacePolicyOperation(namespaceName, PolicyName.AUTO_TOPIC_CREATION, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); if (autoTopicCreationOverride != null) { - if (!AutoTopicCreationOverride.isValidOverride(autoTopicCreationOverride)) { + if (!AutoTopicCreationOverrideImpl.isValidOverride(autoTopicCreationOverride)) { throw new RestException(Status.PRECONDITION_FAILED, "Invalid configuration for autoTopicCreationOverride"); } - if (maxPartitions > 0 && autoTopicCreationOverride.defaultNumPartitions > maxPartitions) { + if (maxPartitions > 0 && autoTopicCreationOverride.getDefaultNumPartitions() > maxPartitions) { throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be less than or equal to " + maxPartitions); } @@ -812,7 +814,7 @@ protected void internalSetAutoTopicCreation(AsyncResponse asyncResponse, return policies; }).thenApply(r -> { String autoOverride = (autoTopicCreationOverride != null - && autoTopicCreationOverride.allowAutoTopicCreation) ? "enabled" : "disabled"; + && autoTopicCreationOverride.isAllowAutoTopicCreation()) ? "enabled" : "disabled"; log.info("[{}] Successfully {} autoTopicCreation on namespace {}", clientAppId(), autoOverride != null ? autoOverride : "removed", namespaceName); asyncResponse.resume(Response.noContent().build()); @@ -844,7 +846,7 @@ protected void internalSetAutoSubscriptionCreation( return policies; }).thenApply(r -> { if (autoSubscriptionCreationOverride != null) { - String autoOverride = autoSubscriptionCreationOverride.allowAutoSubscriptionCreation ? "enabled" + String autoOverride = autoSubscriptionCreationOverride.isAllowAutoSubscriptionCreation() ? "enabled" : "disabled"; log.info("[{}] Successfully {} autoSubscriptionCreation on namespace {}", clientAppId(), autoOverride != null ? autoOverride : "removed", namespaceName); @@ -1184,7 +1186,7 @@ protected PublishRate internalGetPublishRate() { } @SuppressWarnings("deprecation") - protected void internalSetTopicDispatchRate(DispatchRate dispatchRate) { + protected void internalSetTopicDispatchRate(DispatchRateImpl dispatchRate) { validateSuperUserAccess(); log.info("[{}] Set namespace dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); @@ -1230,7 +1232,7 @@ protected DispatchRate internalGetTopicDispatchRate() { return policies.topicDispatchRate.get(pulsar().getConfiguration().getClusterName()); } - protected void internalSetSubscriptionDispatchRate(DispatchRate dispatchRate) { + protected void internalSetSubscriptionDispatchRate(DispatchRateImpl dispatchRate) { validateSuperUserAccess(); log.info("[{}] Set namespace subscription dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); @@ -1332,7 +1334,7 @@ protected void internalRemoveReplicatorDispatchRate() { } } - protected void internalSetReplicatorDispatchRate(DispatchRate dispatchRate) { + protected void internalSetReplicatorDispatchRate(DispatchRateImpl dispatchRate) { validateSuperUserAccess(); log.info("[{}] Set namespace replicator dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); try { @@ -1941,7 +1943,7 @@ private void unsubscribe(NamespaceName nsName, String bundleRange, String subscr */ private void validatePeerClusterConflict(String clusterName, Set replicationClusters) { try { - ClusterDataImpl clusterData = clusterResources().get(path("clusters", clusterName)).orElseThrow( + ClusterData clusterData = clusterResources().get(path("clusters", clusterName)).orElseThrow( () -> new RestException(Status.PRECONDITION_FAILED, "Invalid replication cluster " + clusterName)); Set peerClusters = clusterData.getPeerClusterNames(); if (peerClusters != null && !peerClusters.isEmpty()) { @@ -1980,7 +1982,10 @@ protected BundlesData validateBundlesData(BundlesData initialBundles) { } List bundles = Lists.newArrayList(); bundles.addAll(partitions); - return new BundlesData(bundles); + return BundlesData.builder() + .boundaries(bundles) + .numBundles(bundles.size() - 1) + .build(); } private void validatePolicies(NamespaceName ns, Policies policies) { @@ -2266,7 +2271,7 @@ protected void internalSetOffloadThreshold(long newThreshold) { if (policies.offload_policies == null) { policies.offload_policies = new OffloadPoliciesImpl(); } - policies.offload_policies.setManagedLedgerOffloadThresholdInBytes(newThreshold); + ((OffloadPoliciesImpl) policies.offload_policies).setManagedLedgerOffloadThresholdInBytes(newThreshold); policies.offload_threshold = newThreshold; return policies; }); @@ -2301,7 +2306,8 @@ protected void internalSetOffloadDeletionLag(Long newDeletionLagMs) { if (policies.offload_policies == null) { policies.offload_policies = new OffloadPoliciesImpl(); } - policies.offload_policies.setManagedLedgerOffloadDeletionLagInMillis(newDeletionLagMs); + ((OffloadPoliciesImpl) policies.offload_policies) + .setManagedLedgerOffloadDeletionLagInMillis(newDeletionLagMs); policies.offload_deletion_lag_ms = newDeletionLagMs; return policies; }); 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 dfa8e16f3d2bf..240bef95b0727 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 @@ -107,7 +107,6 @@ import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PartitionedTopicInternalStats; -import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; @@ -121,6 +120,10 @@ import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; +import org.apache.pulsar.common.policies.data.stats.PartitionedTopicStatsImpl; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; @@ -205,13 +208,12 @@ protected Map> internalGetPermissionsOnTopic() { AuthPolicies auth = policies.auth_policies; // First add namespace level permissions - for (String role : auth.namespace_auth.keySet()) { - permissions.put(role, auth.namespace_auth.get(role)); - } + auth.getNamespaceAuthentication().forEach(permissions::put); // Then add topic level permissions - if (auth.destination_auth.containsKey(topicUri)) { - for (Map.Entry> entry : auth.destination_auth.get(topicUri).entrySet()) { + if (auth.getTopicAuthentication().containsKey(topicUri)) { + for (Map.Entry> entry : + auth.getTopicAuthentication().get(topicUri).entrySet()) { String role = entry.getKey(); Set topicPermissions = entry.getValue(); @@ -257,11 +259,11 @@ public void validateAdminOperationOnTopic(boolean authoritative) { private void grantPermissions(String topicUri, String role, Set actions) { try { namespaceResources().set(path(POLICIES, namespaceName.toString()), (policies) -> { - if (!policies.auth_policies.destination_auth.containsKey(topicUri)) { - policies.auth_policies.destination_auth.put(topicUri, new HashMap<>()); + if (!policies.auth_policies.getTopicAuthentication().containsKey(topicUri)) { + policies.auth_policies.getTopicAuthentication().put(topicUri, new HashMap<>()); } - policies.auth_policies.destination_auth.get(topicUri).put(role, actions); + policies.auth_policies.getTopicAuthentication().get(topicUri).put(role, actions); return policies; }); log.info("[{}] Successfully granted access for role {}: {} - topic {}", clientAppId(), role, actions, @@ -316,8 +318,8 @@ private void revokePermissions(String topicUri, String role) { log.error("[{}] Failed to revoke permissions for topic {}", clientAppId(), topicUri, e); throw new RestException(e); } - if (!policies.auth_policies.destination_auth.containsKey(topicUri) - || !policies.auth_policies.destination_auth.get(topicUri).containsKey(role)) { + if (!policies.auth_policies.getTopicAuthentication().containsKey(topicUri) + || !policies.auth_policies.getTopicAuthentication().get(topicUri).containsKey(role)) { log.warn("[{}] Failed to revoke permission from role {} on topic: Not set at topic level {}", clientAppId(), role, topicUri); throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the topic level"); @@ -326,7 +328,7 @@ private void revokePermissions(String topicUri, String role) { // Write the new policies to zookeeper String namespacePath = path(POLICIES, namespaceName.toString()); namespaceResources().set(namespacePath, (p) -> { - p.auth_policies.destination_auth.get(topicUri).remove(role); + p.auth_policies.getTopicAuthentication().get(topicUri).remove(role); return p; }); log.info("[{}] Successfully revoke access for role {} - topic {}", clientAppId(), role, topicUri); @@ -754,16 +756,18 @@ protected CompletableFuture internalGetDelayedDeliveryP TopicPolicies policies = getTopicPolicies(topicName).orElseGet(TopicPolicies::new); DelayedDeliveryPolicies delayedDeliveryPolicies = null; if (policies.isDelayedDeliveryEnabledSet() && policies.isDelayedDeliveryTickTimeMillisSet()) { - delayedDeliveryPolicies = new DelayedDeliveryPolicies( - policies.getDelayedDeliveryTickTimeMillis(), - policies.getDelayedDeliveryEnabled()); + delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(policies.getDelayedDeliveryTickTimeMillis()) + .active(policies.getDelayedDeliveryEnabled()) + .build(); } if (delayedDeliveryPolicies == null && applied) { delayedDeliveryPolicies = getNamespacePolicies(namespaceName).delayed_delivery_policies; if (delayedDeliveryPolicies == null) { - delayedDeliveryPolicies = new DelayedDeliveryPolicies( - pulsar().getConfiguration().getDelayedDeliveryTickTimeMillis(), - pulsar().getConfiguration().isDelayedDeliveryEnabled()); + delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(pulsar().getConfiguration().getDelayedDeliveryTickTimeMillis()) + .active(pulsar().getConfiguration().isDelayedDeliveryEnabled()) + .build(); } } return CompletableFuture.completedFuture(delayedDeliveryPolicies); @@ -1244,7 +1248,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned Topic not found")); return; } - PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata); + PartitionedTopicStatsImpl stats = new PartitionedTopicStatsImpl(partitionMetadata); List> topicStatsFutureList = Lists.newArrayList(); for (int i = 0; i < partitionMetadata.partitions; i++) { try { @@ -1279,7 +1283,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean try { boolean zkPathExists = namespaceResources().getPartitionedTopicResources().exists(path); if (zkPathExists) { - stats.partitions.put(topicName.toString(), new TopicStats()); + stats.partitions.put(topicName.toString(), new TopicStatsImpl()); } else { asyncResponse.resume( new RestException(Status.NOT_FOUND, @@ -2551,7 +2555,8 @@ protected Map internalGetBacklogQuo } protected void internalSetBacklogQuota(AsyncResponse asyncResponse, - BacklogQuota.BacklogQuotaType backlogQuotaType, BacklogQuota backlogQuota) { + BacklogQuota.BacklogQuotaType backlogQuotaType, + BacklogQuotaImpl backlogQuota) { validateTopicPolicyOperation(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); if (backlogQuotaType == null) { @@ -2585,7 +2590,7 @@ protected void internalSetBacklogQuota(AsyncResponse asyncResponse, } else { topicPolicies.getBackLogQuotaMap().remove(backlogQuotaType.name()); } - Map backLogQuotaMap = topicPolicies.getBackLogQuotaMap(); + Map backLogQuotaMap = topicPolicies.getBackLogQuotaMap(); pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies) .whenComplete((r, ex) -> { if (ex != null) { @@ -2823,12 +2828,12 @@ protected CompletableFuture internalSetMaxSubscriptionsPerTopic(Integer ma return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies); } - protected CompletableFuture internalGetReplicatorDispatchRate(boolean applied) { - DispatchRate dispatchRate = getTopicPolicies(topicName) + protected CompletableFuture internalGetReplicatorDispatchRate(boolean applied) { + DispatchRateImpl dispatchRate = getTopicPolicies(topicName) .map(TopicPolicies::getReplicatorDispatchRate) .orElseGet(() -> { if (applied) { - DispatchRate namespacePolicy = getNamespacePolicies(namespaceName) + DispatchRateImpl namespacePolicy = getNamespacePolicies(namespaceName) .replicatorDispatchRate.get(pulsar().getConfiguration().getClusterName()); return namespacePolicy == null ? replicatorDispatchRate() : namespacePolicy; } @@ -2837,7 +2842,7 @@ protected CompletableFuture internalGetReplicatorDispatchRate(bool return CompletableFuture.completedFuture(dispatchRate); } - protected CompletableFuture internalSetReplicatorDispatchRate(DispatchRate dispatchRate) { + protected CompletableFuture internalSetReplicatorDispatchRate(DispatchRateImpl dispatchRate) { TopicPolicies topicPolicies = getTopicPolicies(topicName).orElseGet(TopicPolicies::new); topicPolicies.setReplicatorDispatchRate(dispatchRate); return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies); @@ -3534,10 +3539,10 @@ private CompletableFuture createSubscriptions(TopicName topicName, int num admin.topics().getStatsAsync(topicName.getPartition(0).toString()).thenAccept(stats -> { List> subscriptionFutures = new ArrayList<>(); - stats.subscriptions.entrySet().forEach(e -> { + stats.getSubscriptions().entrySet().forEach(e -> { String subscription = e.getKey(); SubscriptionStats ss = e.getValue(); - if (!ss.isDurable) { + if (!ss.isDurable()) { // We must not re-create non-durable subscriptions on the new partitions return; } @@ -3747,12 +3752,12 @@ protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean aut } - protected CompletableFuture internalGetDispatchRate(boolean applied) { - DispatchRate dispatchRate = getTopicPolicies(topicName) + protected CompletableFuture internalGetDispatchRate(boolean applied) { + DispatchRateImpl dispatchRate = getTopicPolicies(topicName) .map(TopicPolicies::getDispatchRate) .orElseGet(() -> { if (applied) { - DispatchRate namespacePolicy = getNamespacePolicies(namespaceName) + DispatchRateImpl namespacePolicy = getNamespacePolicies(namespaceName) .topicDispatchRate.get(pulsar().getConfiguration().getClusterName()); return namespacePolicy == null ? dispatchRate() : namespacePolicy; } @@ -3761,7 +3766,7 @@ protected CompletableFuture internalGetDispatchRate(boolean applie return CompletableFuture.completedFuture(dispatchRate); } - protected CompletableFuture internalSetDispatchRate(DispatchRate dispatchRate) { + protected CompletableFuture internalSetDispatchRate(DispatchRateImpl dispatchRate) { if (dispatchRate == null) { return CompletableFuture.completedFuture(null); } @@ -3786,7 +3791,7 @@ protected CompletableFuture internalGetSubscriptionDispatchRate(bo .map(TopicPolicies::getSubscriptionDispatchRate) .orElseGet(() -> { if (applied) { - DispatchRate namespacePolicy = getNamespacePolicies(namespaceName) + DispatchRateImpl namespacePolicy = getNamespacePolicies(namespaceName) .subscriptionDispatchRate.get(pulsar().getConfiguration().getClusterName()); return namespacePolicy == null ? subscriptionDispatchRate() : namespacePolicy; } @@ -3795,7 +3800,7 @@ protected CompletableFuture internalGetSubscriptionDispatchRate(bo return CompletableFuture.completedFuture(dispatchRate); } - protected CompletableFuture internalSetSubscriptionDispatchRate(DispatchRate dispatchRate) { + protected CompletableFuture internalSetSubscriptionDispatchRate(DispatchRateImpl dispatchRate) { if (dispatchRate == null) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java index df42b0dc65de7..d81da0a4153e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.Constants; import org.apache.pulsar.common.naming.NamedEntity; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; @@ -197,7 +198,7 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, asyncResponse.resume(new RestException(Status.NOT_FOUND, "Tenant " + tenant + " not found")); return; } - TenantInfoImpl oldTenantAdmin = tenantAdmin.get(); + TenantInfo oldTenantAdmin = tenantAdmin.get(); Set newClusters = new HashSet<>(newTenantAdmin.getAllowedClusters()); canUpdateCluster(tenant, oldTenantAdmin.getAllowedClusters(), newClusters).thenApply(r -> { tenantResources().setAsync(path(POLICIES, tenant), old -> { @@ -334,7 +335,7 @@ protected void internalDeleteTenantForcefully(AsyncResponse asyncResponse, Strin }); } - private void validateClusters(TenantInfoImpl info) { + private void validateClusters(TenantInfo info) { // empty cluster shouldn't be allowed if (info == null || info.getAllowedClusters().stream().filter(c -> !StringUtils.isBlank(c)) .collect(Collectors.toSet()).isEmpty() diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java index bcc989269c42d..3e7067ba92cdd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java @@ -67,6 +67,7 @@ import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; import org.apache.pulsar.common.policies.data.TenantOperation; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -241,7 +242,7 @@ public Map> getPermissions(@PathParam("property") String validateNamespaceOperation(NamespaceName.get(property, namespace), NamespaceOperation.GET_PERMISSION); Policies policies = getNamespacePolicies(namespaceName); - return policies.auth_policies.namespace_auth; + return policies.auth_policies.getNamespaceAuthentication(); } @POST @@ -627,7 +628,7 @@ public PublishRate getPublishRate(@PathParam("property") String property, @PathP @ApiOperation(hidden = true, value = "Set dispatch-rate throttling for all topics of the namespace") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, DispatchRate dispatchRate) { + @PathParam("namespace") String namespace, DispatchRateImpl dispatchRate) { validateNamespaceName(property, cluster, namespace); internalSetTopicDispatchRate(dispatchRate); } @@ -651,7 +652,7 @@ public DispatchRate getDispatchRate(@PathParam("property") String property, @Pat public void setSubscriptionDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - DispatchRate dispatchRate) { + DispatchRateImpl dispatchRate) { validateNamespaceName(property, cluster, namespace); internalSetSubscriptionDispatchRate(dispatchRate); } @@ -677,7 +678,7 @@ public void setReplicatorDispatchRate( @PathParam("tenant") String tenant, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, @ApiParam(value = "Replicator dispatch rate for all topics of the specified namespace") - DispatchRate dispatchRate) { + DispatchRateImpl dispatchRate) { validateNamespaceName(tenant, cluster, namespace); internalSetReplicatorDispatchRate(dispatchRate); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java index 6b4e5a581c205..8eae28205b97f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java @@ -95,7 +95,7 @@ public BookiesClusterInfo getAllBookies() throws Exception { RawBookieInfo bookieInfo = new RawBookieInfo(bookieId.toString()); result.add(bookieInfo); } - return new BookiesClusterInfo(result); + return BookiesClusterInfo.builder().bookies(result).build(); } @GET 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 85a6ac67a8702..4609377913f9b 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 @@ -68,6 +68,7 @@ import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -188,7 +189,7 @@ public Map> getPermissions(@PathParam("tenant") String t validateNamespaceOperation(NamespaceName.get(tenant, namespace), NamespaceOperation.GET_PERMISSION); Policies policies = getNamespacePolicies(namespaceName); - return policies.auth_policies.namespace_auth; + return policies.auth_policies.getNamespaceAuthentication(); } @POST @@ -568,7 +569,8 @@ public PublishRate getPublishRate( @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setDispatchRate(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, - @ApiParam(value = "Dispatch rate for all topics of the specified namespace") DispatchRate dispatchRate) { + @ApiParam(value = "Dispatch rate for all topics of the specified namespace") + DispatchRateImpl dispatchRate) { validateNamespaceName(tenant, namespace); internalSetTopicDispatchRate(dispatchRate); } @@ -600,7 +602,7 @@ public DispatchRate getDispatchRate(@PathParam("tenant") String tenant, public void setSubscriptionDispatchRate(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Subscription dispatch rate for all topics of the specified namespace") - DispatchRate dispatchRate) { + DispatchRateImpl dispatchRate) { validateNamespaceName(tenant, namespace); internalSetSubscriptionDispatchRate(dispatchRate); } @@ -675,7 +677,7 @@ public void removeReplicatorDispatchRate( public void setReplicatorDispatchRate(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Replicator dispatch rate for all topics of the specified namespace") - DispatchRate dispatchRate) { + DispatchRateImpl dispatchRate) { validateNamespaceName(tenant, namespace); internalSetReplicatorDispatchRate(dispatchRate); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index ce8eada750e71..97727575ddffc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -59,7 +59,6 @@ import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; -import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PersistencePolicies; @@ -72,6 +71,8 @@ import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1488,7 +1489,7 @@ public void setBacklogQuota( @Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, - @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, BacklogQuota backlogQuota) { + @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, BacklogQuotaImpl backlogQuota) { validateTopicName(tenant, namespace, encodedTopic); preValidation(); internalSetBacklogQuota(asyncResponse, backlogQuotaType, backlogQuota); @@ -1963,7 +1964,7 @@ public void setReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "Replicator dispatch rate of the topic") - DispatchRate dispatchRate) { + DispatchRateImpl dispatchRate) { validateTopicName(tenant, namespace, encodedTopic); preValidation(); internalSetReplicatorDispatchRate(dispatchRate).whenComplete((r, ex) -> { @@ -2521,7 +2522,8 @@ public void setDispatchRate(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, - @ApiParam(value = "Dispatch rate for the specified topic") DispatchRate dispatchRate) { + @ApiParam(value = "Dispatch rate for the specified topic") + DispatchRateImpl dispatchRate) { validateTopicName(tenant, namespace, encodedTopic); preValidation(); internalSetDispatchRate(dispatchRate).whenComplete((r, ex) -> { @@ -2617,7 +2619,7 @@ public void setSubscriptionDispatchRate( @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "Subscription message dispatch rate for the specified topic") - DispatchRate dispatchRate) { + DispatchRateImpl dispatchRate) { validateTopicName(tenant, namespace, encodedTopic); preValidation(); internalSetSubscriptionDispatchRate(dispatchRate).whenComplete((r, ex) -> { 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 a711a805f5915..682fbc0c0d533 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 @@ -31,7 +31,7 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -406,11 +406,11 @@ protected void aggregateResourceGroupLocalUsages() { long mSecsSinceEpochStart, mSecsSinceEpochEnd, diffMSecs; mSecsSinceEpochStart = System.currentTimeMillis(); BrokerService bs = this.pulsar.getBrokerService(); - Map topicStatsMap = bs.getTopicStats(); + Map topicStatsMap = bs.getTopicStats(); - for (Map.Entry entry : topicStatsMap.entrySet()) { + for (Map.Entry entry : topicStatsMap.entrySet()) { String topicName = entry.getKey(); - TopicStats topicStats = entry.getValue(); + TopicStatsImpl topicStats = entry.getValue(); TopicName topic = TopicName.get(topicName); String tenantString = topic.getTenant(); String nsString = topic.getNamespacePortion(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index 3862757596d59..21792b3e232f3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -37,6 +37,7 @@ import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TopicPolicies; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.zookeeper.ZooKeeperDataCache; import org.slf4j.Logger; @@ -44,7 +45,7 @@ public class BacklogQuotaManager { private static final Logger log = LoggerFactory.getLogger(BacklogQuotaManager.class); - private final BacklogQuota defaultQuota; + private final BacklogQuotaImpl defaultQuota; private final ZooKeeperDataCache zkCache; private final PulsarService pulsar; private final boolean isTopicLevelPoliciesEnable; @@ -52,22 +53,24 @@ public class BacklogQuotaManager { public BacklogQuotaManager(PulsarService pulsar) { this.isTopicLevelPoliciesEnable = pulsar.getConfiguration().isTopicLevelPoliciesEnabled(); - this.defaultQuota = new BacklogQuota( - pulsar.getConfiguration().getBacklogQuotaDefaultLimitGB() * 1024 * 1024 * 1024, - pulsar.getConfiguration().getBacklogQuotaDefaultLimitSecond(), - pulsar.getConfiguration().getBacklogQuotaDefaultRetentionPolicy()); + this.defaultQuota = BacklogQuotaImpl.builder() + .limitSize(pulsar.getConfiguration().getBacklogQuotaDefaultLimitGB() * 1024 * 1024 * 1024) + .limitTime(pulsar.getConfiguration().getBacklogQuotaDefaultLimitSecond()) + .retentionPolicy(pulsar.getConfiguration().getBacklogQuotaDefaultRetentionPolicy()) + .build(); this.zkCache = pulsar.getConfigurationCache().policiesCache(); this.pulsar = pulsar; } - public BacklogQuota getDefaultQuota() { + public BacklogQuotaImpl getDefaultQuota() { return this.defaultQuota; } - public BacklogQuota getBacklogQuota(String namespace, String policyPath) { + public BacklogQuotaImpl getBacklogQuota(String namespace, String policyPath) { try { return zkCache.get(policyPath) - .map(p -> p.backlog_quota_map.getOrDefault(BacklogQuotaType.destination_storage, defaultQuota)) + .map(p -> (BacklogQuotaImpl) p.backlog_quota_map + .getOrDefault(BacklogQuotaType.destination_storage, defaultQuota)) .orElse(defaultQuota); } catch (Exception e) { log.warn("Failed to read policies data, will apply the default backlog quota: namespace={}", namespace, e); @@ -75,7 +78,7 @@ public BacklogQuota getBacklogQuota(String namespace, String policyPath) { } } - public BacklogQuota getBacklogQuota(TopicName topicName) { + public BacklogQuotaImpl getBacklogQuota(TopicName topicName) { String policyPath = AdminResource.path(POLICIES, topicName.getNamespace()); if (!isTopicLevelPoliciesEnable) { return getBacklogQuota(topicName.getNamespace(), policyPath); 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 ddca6b2cdd71d..6089f11a09484 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 @@ -151,8 +151,8 @@ import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TopicPolicies; -import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FieldParser; @@ -1346,9 +1346,9 @@ public CompletableFuture getManagedLedgerConfig(TopicName t .setBookKeeperEnsemblePlacementPolicyClassName(ZkIsolatedBookieEnsemblePlacementPolicy.class); Map properties = Maps.newHashMap(); properties.put(ZkIsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.bookkeeperAffinityGroupPrimary); + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); properties.put(ZkIsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.bookkeeperAffinityGroupSecondary); + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); } managedLedgerConfig.setThrottleMarkDelete(persistencePolicies.getManagedLedgerMaxMarkDeleteRate()); @@ -1835,8 +1835,8 @@ public String generateUniqueProducerName() { return producerNameGenerator.getNextId(); } - public Map getTopicStats() { - HashMap stats = new HashMap<>(); + public Map getTopicStats() { + HashMap stats = new HashMap<>(); forEachTopic(topic -> stats.put(topic.getName(), topic.getStats(false, false))); @@ -2496,7 +2496,7 @@ public boolean isAllowAutoTopicCreation(final TopicName topicName) { } AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName); if (autoTopicCreationOverride != null) { - return autoTopicCreationOverride.allowAutoTopicCreation; + return autoTopicCreationOverride.isAllowAutoTopicCreation(); } else { return pulsar.getConfiguration().isAllowAutoTopicCreation(); } @@ -2505,7 +2505,7 @@ public boolean isAllowAutoTopicCreation(final TopicName topicName) { public boolean isDefaultTopicTypePartitioned(final TopicName topicName) { AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName); if (autoTopicCreationOverride != null) { - return TopicType.PARTITIONED.toString().equals(autoTopicCreationOverride.topicType); + return TopicType.PARTITIONED.toString().equals(autoTopicCreationOverride.getTopicType()); } else { return pulsar.getConfiguration().isDefaultTopicTypePartitioned(); } @@ -2514,7 +2514,7 @@ public boolean isDefaultTopicTypePartitioned(final TopicName topicName) { public int getDefaultNumPartitions(final TopicName topicName) { AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName); if (autoTopicCreationOverride != null) { - return autoTopicCreationOverride.defaultNumPartitions; + return autoTopicCreationOverride.getDefaultNumPartitions(); } else { return pulsar.getConfiguration().getDefaultNumPartitions(); } @@ -2547,7 +2547,7 @@ public boolean isAllowAutoSubscriptionCreation(final TopicName topicName) { AutoSubscriptionCreationOverride autoSubscriptionCreationOverride = getAutoSubscriptionCreationOverride(topicName); if (autoSubscriptionCreationOverride != null) { - return autoSubscriptionCreationOverride.allowAutoSubscriptionCreation; + return autoSubscriptionCreationOverride.isAllowAutoSubscriptionCreation(); } else { return pulsar.getConfiguration().isAllowAutoSubscriptionCreation(); } 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 f905cb52fd377..f093cd66440fc 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 @@ -50,7 +50,7 @@ import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.api.proto.MessageIdData; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.stats.Rate; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; @@ -98,7 +98,7 @@ public class Consumer { private final ConcurrentLongLongPairHashMap pendingAcks; - private final ConsumerStats stats; + private final ConsumerStatsImpl stats; private volatile int maxUnackedMessages; private static final AtomicIntegerFieldUpdater UNACKED_MESSAGES_UPDATER = @@ -156,7 +156,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.metadata = metadata != null ? metadata : Collections.emptyMap(); - stats = new ConsumerStats(); + stats = new ConsumerStatsImpl(); if (cnx.hasHAProxyMessage()) { stats.setAddress(cnx.getHAProxyMessage().sourceAddress() + ":" + cnx.getHAProxyMessage().sourcePort()); } else { @@ -581,7 +581,7 @@ public void reachedEndOfTopic() { /** * Checks if consumer-blocking on unAckedMessages is allowed for below conditions:
* a. consumer must have Shared-subscription
- * b. {@link maxUnackedMessages} value > 0 + * b. {@link this#maxUnackedMessages} value > 0 * * @return */ @@ -596,11 +596,10 @@ public void updateRates() { stats.msgRateOut = msgOut.getRate(); stats.msgThroughputOut = msgOut.getValueRate(); stats.msgRateRedeliver = msgRedeliver.getRate(); - stats.chuckedMessageRate = chunkedMessageRate.getRate(); stats.chunkedMessageRate = chunkedMessageRate.getRate(); } - public void updateStats(ConsumerStats consumerStats) { + public void updateStats(ConsumerStatsImpl consumerStats) { msgOutCounter.add(consumerStats.msgOutCounter); bytesOutCounter.add(consumerStats.bytesOutCounter); msgOut.recordMultipleEvents(consumerStats.msgOutCounter, consumerStats.bytesOutCounter); @@ -616,7 +615,7 @@ public void updateStats(ConsumerStats consumerStats) { AVG_MESSAGES_PER_ENTRY.set(this, consumerStats.avgMessagesPerEntry); } - public ConsumerStats getStats() { + public ConsumerStatsImpl getStats() { stats.msgOutCounter = msgOutCounter.longValue(); stats.bytesOutCounter = bytesOutCounter.longValue(); stats.lastAckedTimestamp = lastAckedTimestamp; 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 d9ccb0f863f8e..15076f9dc3550 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 @@ -45,8 +45,8 @@ import org.apache.pulsar.common.api.proto.ProducerAccessMode; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; -import org.apache.pulsar.common.policies.data.PublisherStats; +import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.stats.Rate; @@ -77,7 +77,7 @@ public class Producer { private boolean isClosed = false; private final CompletableFuture closeFuture; - private final PublisherStats stats; + private final PublisherStatsImpl stats; private final boolean isRemote; private final String remoteCluster; private final boolean isNonPersistentTopic; @@ -111,7 +111,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN this.metadata = metadata != null ? metadata : Collections.emptyMap(); - this.stats = isNonPersistentTopic ? new NonPersistentPublisherStats() : new PublisherStats(); + this.stats = isNonPersistentTopic ? new NonPersistentPublisherStatsImpl() : new PublisherStatsImpl(); if (cnx.hasHAProxyMessage()) { stats.setAddress(cnx.getHAProxyMessage().sourceAddress() + ":" + cnx.getHAProxyMessage().sourcePort()); } else { @@ -583,7 +583,7 @@ public void updateRates() { } if (this.isNonPersistentTopic) { msgDrop.calculateRate(); - ((NonPersistentPublisherStats) stats).msgDropRate = msgDrop.getRate(); + ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getRate(); } } @@ -599,7 +599,7 @@ public String getRemoteCluster() { return remoteCluster; } - public PublisherStats getStats() { + public PublisherStatsImpl getStats() { return stats; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java index 2d532b969c623..5f738ac193706 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java @@ -22,13 +22,13 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.ReplicatorStats; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; public interface Replicator { void startProducer(); - ReplicatorStats getStats(); + ReplicatorStatsImpl getStats(); CompletableFuture disconnect(); 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 757f887fafc6b..2f3d32d1f9c19 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 @@ -128,8 +128,8 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; -import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.CommandUtils; import org.apache.pulsar.common.protocol.Commands; @@ -566,7 +566,7 @@ protected void handleConsumerStats(CommandConsumerStats commandConsumerStats) { } ByteBuf createConsumerStatsResponse(Consumer consumer, long requestId) { - ConsumerStats consumerStats = consumer.getStats(); + ConsumerStatsImpl consumerStats = consumer.getStats(); Subscription subscription = consumer.getSubscription(); BaseCommand cmd = Commands.newConsumerStatsResponseCommand(ServerError.UnknownError, null, requestId); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StreamingStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StreamingStats.java index e2b2f17da9695..02dcb8233ffae 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StreamingStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StreamingStats.java @@ -19,14 +19,14 @@ package org.apache.pulsar.broker.service; import org.apache.pulsar.common.api.proto.CommandSubscribe; -import org.apache.pulsar.common.policies.data.ConsumerStats; -import org.apache.pulsar.common.policies.data.PublisherStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; import org.apache.pulsar.utils.StatsOutputStream; public class StreamingStats { private StreamingStats() {} - public static void writePublisherStats(StatsOutputStream statsStream, PublisherStats stats) { + public static void writePublisherStats(StatsOutputStream statsStream, PublisherStatsImpl stats) { statsStream.startObject(); statsStream.writePair("msgRateIn", stats.msgRateIn); @@ -53,7 +53,7 @@ public static void writePublisherStats(StatsOutputStream statsStream, PublisherS public static void writeConsumerStats(StatsOutputStream statsStream, CommandSubscribe.SubType subType, - ConsumerStats stats) { + ConsumerStatsImpl stats) { // Populate consumer specific stats here statsStream.startObject(); 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 6b9160c7726c3..d1e4506fb6422 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 @@ -35,7 +35,7 @@ import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -194,7 +194,7 @@ void updateRates(NamespaceStats nsStats, NamespaceBundleStats currentBundleStats ConcurrentOpenHashMap getReplicators(); - TopicStats getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize); + TopicStatsImpl getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize); CompletableFuture getInternalStats(boolean includeLedgerMetadata); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index 85f631d3434e7..b003db823ed4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -35,7 +35,7 @@ import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.SendCallback; -import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats; +import org.apache.pulsar.common.policies.data.stats.NonPersistentReplicatorStatsImpl; import org.apache.pulsar.common.stats.Rate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +45,7 @@ public class NonPersistentReplicator extends AbstractReplicator implements Repli private final Rate msgOut = new Rate(); private final Rate msgDrop = new Rate(); - private final NonPersistentReplicatorStats stats = new NonPersistentReplicatorStats(); + private final NonPersistentReplicatorStatsImpl stats = new NonPersistentReplicatorStatsImpl(); public NonPersistentReplicator(NonPersistentTopic topic, String localCluster, String remoteCluster, BrokerService brokerService) throws NamingException { @@ -134,7 +134,7 @@ public void updateRates() { } @Override - public NonPersistentReplicatorStats getStats() { + public NonPersistentReplicatorStatsImpl getStats() { stats.connected = producer != null && producer.isConnected(); stats.replicationDelayInSeconds = getReplicationDelayInSeconds(); 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 c16065dddeb52..c605f2388af79 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 @@ -45,8 +45,8 @@ import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.ConsumerStats; -import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentSubscriptionStatsImpl; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -197,7 +197,7 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor dispatcher.removeConsumer(consumer); } // preserve accumulative stats form removed consumer - ConsumerStats stats = consumer.getStats(); + ConsumerStatsImpl stats = consumer.getStats(); bytesOutFromRemovedConsumers.add(stats.bytesOutCounter); msgOutFromRemovedConsumer.add(stats.msgOutCounter); @@ -452,15 +452,15 @@ public boolean expireMessages(Position position) { + " non-persistent topic."); } - public NonPersistentSubscriptionStats getStats() { - NonPersistentSubscriptionStats subStats = new NonPersistentSubscriptionStats(); + public NonPersistentSubscriptionStatsImpl getStats() { + NonPersistentSubscriptionStatsImpl subStats = new NonPersistentSubscriptionStatsImpl(); subStats.bytesOutCounter = bytesOutFromRemovedConsumers.longValue(); subStats.msgOutCounter = msgOutFromRemovedConsumer.longValue(); NonPersistentDispatcher dispatcher = this.dispatcher; if (dispatcher != null) { dispatcher.getConsumers().forEach(consumer -> { - ConsumerStats consumerStats = consumer.getStats(); + ConsumerStatsImpl consumerStats = consumer.getStats(); subStats.consumers.add(consumerStats); subStats.msgRateOut += consumerStats.msgRateOut; subStats.msgThroughputOut += consumerStats.msgThroughputOut; 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 90f2fb7d42fbb..464cd8e6417cc 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 @@ -67,16 +67,17 @@ import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; -import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats; -import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; -import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats; -import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats; -import org.apache.pulsar.common.policies.data.NonPersistentTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PublisherStats; -import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentSubscriptionStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -647,7 +648,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsStream.startList("publishers"); producers.values().forEach(producer -> { producer.updateRates(); - PublisherStats publisherStats = producer.getStats(); + PublisherStatsImpl publisherStats = producer.getStats(); topicStats.aggMsgRateIn += publisherStats.msgRateIn; topicStats.aggMsgThroughputIn += publisherStats.msgThroughputIn; @@ -689,7 +690,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats consumer.updateRates(); - ConsumerStats consumerStats = consumer.getStats(); + ConsumerStatsImpl consumerStats = consumer.getStats(); subMsgRateOut += consumerStats.msgRateOut; subMsgThroughputOut += consumerStats.msgThroughputOut; subMsgRateRedeliver += consumerStats.msgRateRedeliver; @@ -761,14 +762,14 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats } @Override - public NonPersistentTopicStats getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize) { + public NonPersistentTopicStatsImpl getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize) { - NonPersistentTopicStats stats = new NonPersistentTopicStats(); + NonPersistentTopicStatsImpl stats = new NonPersistentTopicStatsImpl(); - ObjectObjectHashMap remotePublishersStats = new ObjectObjectHashMap<>(); + ObjectObjectHashMap remotePublishersStats = new ObjectObjectHashMap<>(); producers.values().forEach(producer -> { - NonPersistentPublisherStats publisherStats = (NonPersistentPublisherStats) producer.getStats(); + NonPersistentPublisherStatsImpl publisherStats = (NonPersistentPublisherStatsImpl) producer.getStats(); stats.msgRateIn += publisherStats.msgRateIn; stats.msgThroughputIn += publisherStats.msgThroughputIn; @@ -787,7 +788,7 @@ public NonPersistentTopicStats getStats(boolean getPreciseBacklog, boolean subsc stats.msgOutCounter = msgOutFromRemovedSubscriptions.longValue(); subscriptions.forEach((name, subscription) -> { - NonPersistentSubscriptionStats subStats = subscription.getStats(); + NonPersistentSubscriptionStatsImpl subStats = subscription.getStats(); stats.msgRateOut += subStats.msgRateOut; stats.msgThroughputOut += subStats.msgThroughputOut; @@ -797,10 +798,10 @@ public NonPersistentTopicStats getStats(boolean getPreciseBacklog, boolean subsc }); replicators.forEach((cluster, replicator) -> { - NonPersistentReplicatorStats replicatorStats = replicator.getStats(); + NonPersistentReplicatorStatsImpl replicatorStats = replicator.getStats(); // Add incoming msg rates - PublisherStats pubStats = remotePublishersStats.get(replicator.getRemoteCluster()); + PublisherStatsImpl pubStats = remotePublishersStats.get(replicator.getRemoteCluster()); if (pubStats != null) { replicatorStats.msgRateIn = pubStats.msgRateIn; replicatorStats.msgThroughputIn = pubStats.msgThroughputIn; @@ -984,7 +985,7 @@ public CompletableFuture unsubscribe(String subscriptionName) { return CompletableFuture.runAsync(() -> { NonPersistentSubscription sub = subscriptions.remove(subscriptionName); // preserve accumulative stats form removed subscription - SubscriptionStats stats = sub.getStats(); + SubscriptionStatsImpl stats = sub.getStats(); bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); }, brokerService.executor()); 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 a3333cd4d4f8b..8cf4427ce4d22 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 @@ -33,6 +33,7 @@ import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TopicPolicies; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,8 +139,13 @@ private DispatchRate createDispatchRate() { dispatchThrottlingRateInByte = -1; } - return new DispatchRate(dispatchThrottlingRateInMsg, dispatchThrottlingRateInByte, 1, - config.isDispatchThrottlingRateRelativeToPublishRate()); + + return DispatchRate.builder() + .dispatchThrottlingRateInMsg(dispatchThrottlingRateInMsg) + .dispatchThrottlingRateInByte(dispatchThrottlingRateInByte) + .ratePeriodInSecond(1) + .relativeToPublishRate(config.isDispatchThrottlingRateRelativeToPublishRate()) + .build(); } /** @@ -268,10 +274,12 @@ public void onPoliciesUpdate(Policies data) { } @SuppressWarnings("deprecation") - public static DispatchRate getPoliciesDispatchRate(final String cluster, Optional policies, Type type) { + public static DispatchRateImpl getPoliciesDispatchRate(final String cluster, + Optional policies, + Type type) { // return policy-dispatch rate only if it's enabled in policies return policies.map(p -> { - DispatchRate dispatchRate; + DispatchRateImpl dispatchRate; switch (type) { case TOPIC: dispatchRate = p.topicDispatchRate.get(cluster); @@ -331,11 +339,11 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { // synchronized to prevent race condition from concurrent zk-watch log.info("setting message-dispatch-rate {}", dispatchRate); - long msgRate = dispatchRate.dispatchThrottlingRateInMsg; - long byteRate = dispatchRate.dispatchThrottlingRateInByte; - long ratePeriod = dispatchRate.ratePeriodInSecond; + long msgRate = dispatchRate.getDispatchThrottlingRateInMsg(); + long byteRate = dispatchRate.getDispatchThrottlingRateInByte(); + long ratePeriod = dispatchRate.getRatePeriodInSecond(); - Supplier permitUpdaterMsg = dispatchRate.relativeToPublishRate + Supplier permitUpdaterMsg = dispatchRate.isRelativeToPublishRate() ? () -> getRelativeDispatchRateInMsg(dispatchRate) : null; // update msg-rateLimiter @@ -344,7 +352,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { this.dispatchRateLimiterOnMessage = new RateLimiter(brokerService.pulsar().getExecutor(), msgRate, ratePeriod, TimeUnit.SECONDS, permitUpdaterMsg, true); } else { - this.dispatchRateLimiterOnMessage.setRate(msgRate, dispatchRate.ratePeriodInSecond, + this.dispatchRateLimiterOnMessage.setRate(msgRate, dispatchRate.getRatePeriodInSecond(), TimeUnit.SECONDS, permitUpdaterMsg); } } else { @@ -355,7 +363,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } } - Supplier permitUpdaterByte = dispatchRate.relativeToPublishRate + Supplier permitUpdaterByte = dispatchRate.isRelativeToPublishRate() ? () -> getRelativeDispatchRateInByte(dispatchRate) : null; // update byte-rateLimiter @@ -364,7 +372,7 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { this.dispatchRateLimiterOnByte = new RateLimiter(brokerService.pulsar().getExecutor(), byteRate, ratePeriod, TimeUnit.SECONDS, permitUpdaterByte, true); } else { - this.dispatchRateLimiterOnByte.setRate(byteRate, dispatchRate.ratePeriodInSecond, + this.dispatchRateLimiterOnByte.setRate(byteRate, dispatchRate.getRatePeriodInSecond(), TimeUnit.SECONDS, permitUpdaterByte); } } else { @@ -378,13 +386,13 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { private long getRelativeDispatchRateInMsg(DispatchRate dispatchRate) { return (topic != null && dispatchRate != null) - ? (long) topic.getLastUpdatedAvgPublishRateInMsg() + dispatchRate.dispatchThrottlingRateInMsg + ? (long) topic.getLastUpdatedAvgPublishRateInMsg() + dispatchRate.getDispatchThrottlingRateInMsg() : 0; } private long getRelativeDispatchRateInByte(DispatchRate dispatchRate) { return (topic != null && dispatchRate != null) - ? (long) topic.getLastUpdatedAvgPublishRateInByte() + dispatchRate.dispatchThrottlingRateInByte + ? (long) topic.getLastUpdatedAvgPublishRateInByte() + dispatchRate.getDispatchThrottlingRateInByte() : 0; } @@ -408,8 +416,8 @@ public long getDispatchRateOnByte() { private static boolean isDispatchRateEnabled(DispatchRate dispatchRate) { - return dispatchRate != null && (dispatchRate.dispatchThrottlingRateInMsg > 0 - || dispatchRate.dispatchThrottlingRateInByte > 0); + return dispatchRate != null && (dispatchRate.getDispatchThrottlingRateInMsg() > 0 + || dispatchRate.getDispatchThrottlingRateInByte() > 0); } public void close() { 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 cbdb53a15a8c6..0f3f18c4492de 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 @@ -55,7 +55,7 @@ import org.apache.pulsar.client.impl.SendCallback; import org.apache.pulsar.common.api.proto.MarkerType; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.ReplicatorStats; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.stats.Rate; import org.apache.pulsar.common.util.Codec; import org.slf4j.Logger; @@ -99,7 +99,7 @@ public class PersistentReplicator extends AbstractReplicator // for connected subscriptions, message expiry will be checked if the backlog is greater than this threshold private static final int MINIMUM_BACKLOG_FOR_EXPIRY_CHECK = 1000; - private final ReplicatorStats stats = new ReplicatorStats(); + private final ReplicatorStatsImpl stats = new ReplicatorStatsImpl(); public PersistentReplicator(PersistentTopic topic, ManagedCursor cursor, String localCluster, String remoteCluster, BrokerService brokerService) throws NamingException { @@ -626,7 +626,7 @@ public void updateRates() { stats.msgRateExpired = msgExpired.getRate() + expiryMonitor.getMessageExpiryRate(); } - public ReplicatorStats getStats() { + public ReplicatorStatsImpl getStats() { stats.replicationBacklog = cursor != null ? cursor.getNumberOfEntriesInBacklog(false) : 0; stats.connected = producer != null && producer.isConnected(); stats.replicationDelayInSeconds = getReplicationDelayInSeconds(); 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 1ad24ae8d5cd8..159f39a438b40 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 @@ -70,10 +70,10 @@ import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.apache.pulsar.common.api.proto.TxnAction; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.ConsumerStats; -import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.util.FutureUtil; @@ -283,7 +283,7 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor } // preserve accumulative stats form removed consumer - ConsumerStats stats = consumer.getStats(); + ConsumerStatsImpl stats = consumer.getStats(); bytesOutFromRemovedConsumers.add(stats.bytesOutCounter); msgOutFromRemovedConsumer.add(stats.msgOutCounter); @@ -966,8 +966,8 @@ public long estimateBacklogSize() { return cursor.getEstimatedSizeSinceMarkDeletePosition(); } - public SubscriptionStats getStats(Boolean getPreciseBacklog, boolean subscriptionBacklogSize) { - SubscriptionStats subStats = new SubscriptionStats(); + public SubscriptionStatsImpl getStats(Boolean getPreciseBacklog, boolean subscriptionBacklogSize) { + SubscriptionStatsImpl subStats = new SubscriptionStatsImpl(); subStats.lastExpireTimestamp = lastExpireTimestamp; subStats.lastConsumedFlowTimestamp = lastConsumedFlowTimestamp; subStats.lastMarkDeleteAdvancedTimestamp = lastMarkDeleteAdvancedTimestamp; @@ -978,14 +978,13 @@ public SubscriptionStats getStats(Boolean getPreciseBacklog, boolean subscriptio Map> consumerKeyHashRanges = getType() == SubType.Key_Shared ? ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getConsumerKeyHashRanges() : null; dispatcher.getConsumers().forEach(consumer -> { - ConsumerStats consumerStats = consumer.getStats(); + ConsumerStatsImpl consumerStats = consumer.getStats(); subStats.consumers.add(consumerStats); subStats.msgRateOut += consumerStats.msgRateOut; subStats.msgThroughputOut += consumerStats.msgThroughputOut; subStats.bytesOutCounter += consumerStats.bytesOutCounter; subStats.msgOutCounter += consumerStats.msgOutCounter; subStats.msgRateRedeliver += consumerStats.msgRateRedeliver; - subStats.chuckedMessageRate += consumerStats.chuckedMessageRate; subStats.chunkedMessageRate += consumerStats.chunkedMessageRate; subStats.unackedMessages += consumerStats.unackedMessages; subStats.lastConsumedTimestamp = 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 58273e846b216..c893f8f315af6 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 @@ -121,22 +121,22 @@ import org.apache.pulsar.common.events.EventsTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; -import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.LedgerInfo; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.PublisherStats; -import org.apache.pulsar.common.policies.data.ReplicatorStats; import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicPolicies; -import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.TransactionBufferStats; import org.apache.pulsar.common.policies.data.TransactionInBufferStats; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.protocol.schema.SchemaVersion; @@ -224,7 +224,7 @@ private static class TopicStatsHelper { public double aggMsgThrottlingFailure; public double aggMsgRateOut; public double aggMsgThroughputOut; - public final ObjectObjectHashMap remotePublishersStats; + public final ObjectObjectHashMap remotePublishersStats; public TopicStatsHelper() { remotePublishersStats = new ObjectObjectHashMap<>(); @@ -981,7 +981,7 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { void removeSubscription(String subscriptionName) { PersistentSubscription sub = subscriptions.remove(subscriptionName); // preserve accumulative stats form removed subscription - SubscriptionStats stats = sub.getStats(false, false); + SubscriptionStatsImpl stats = sub.getStats(false, false); bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); } @@ -1531,7 +1531,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsStream.startList("publishers"); producers.values().forEach(producer -> { producer.updateRates(); - PublisherStats publisherStats = producer.getStats(); + PublisherStatsImpl publisherStats = producer.getStats(); topicStatsHelper.aggMsgRateIn += publisherStats.msgRateIn; topicStatsHelper.aggMsgThroughputIn += publisherStats.msgThroughputIn; @@ -1567,10 +1567,10 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats // Update replicator stats - ReplicatorStats rStat = replicator.getStats(); + ReplicatorStatsImpl rStat = replicator.getStats(); // Add incoming msg rates - PublisherStats pubStats = topicStatsHelper.remotePublishersStats.get(replicator.getRemoteCluster()); + PublisherStatsImpl pubStats = topicStatsHelper.remotePublishersStats.get(replicator.getRemoteCluster()); rStat.msgRateIn = pubStats != null ? pubStats.msgRateIn : 0; rStat.msgThroughputIn = pubStats != null ? pubStats.msgThroughputIn : 0; rStat.inboundConnection = pubStats != null ? pubStats.getAddress() : null; @@ -1641,7 +1641,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats ++bundleStats.consumerCount; consumer.updateRates(); - ConsumerStats consumerStats = consumer.getStats(); + ConsumerStatsImpl consumerStats = consumer.getStats(); subMsgRateOut += consumerStats.msgRateOut; subMsgThroughputOut += consumerStats.msgThroughputOut; subMsgRateRedeliver += consumerStats.msgRateRedeliver; @@ -1741,14 +1741,14 @@ public double getLastUpdatedAvgPublishRateInByte() { } @Override - public TopicStats getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize) { + public TopicStatsImpl getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize) { - TopicStats stats = new TopicStats(); + TopicStatsImpl stats = new TopicStatsImpl(); - ObjectObjectHashMap remotePublishersStats = new ObjectObjectHashMap<>(); + ObjectObjectHashMap remotePublishersStats = new ObjectObjectHashMap<>(); producers.values().forEach(producer -> { - PublisherStats publisherStats = producer.getStats(); + PublisherStatsImpl publisherStats = producer.getStats(); stats.msgRateIn += publisherStats.msgRateIn; stats.msgThroughputIn += publisherStats.msgThroughputIn; @@ -1768,7 +1768,7 @@ public TopicStats getStats(boolean getPreciseBacklog, boolean subscriptionBacklo stats.msgOutCounter = msgOutFromRemovedSubscriptions.longValue(); subscriptions.forEach((name, subscription) -> { - SubscriptionStats subStats = subscription.getStats(getPreciseBacklog, subscriptionBacklogSize); + SubscriptionStatsImpl subStats = subscription.getStats(getPreciseBacklog, subscriptionBacklogSize); stats.msgRateOut += subStats.msgRateOut; stats.msgThroughputOut += subStats.msgThroughputOut; @@ -1781,10 +1781,10 @@ public TopicStats getStats(boolean getPreciseBacklog, boolean subscriptionBacklo }); replicators.forEach((cluster, replicator) -> { - ReplicatorStats replicatorStats = replicator.getStats(); + ReplicatorStatsImpl replicatorStats = replicator.getStats(); // Add incoming msg rates - PublisherStats pubStats = remotePublishersStats.get(replicator.getRemoteCluster()); + PublisherStatsImpl pubStats = remotePublishersStats.get(replicator.getRemoteCluster()); if (pubStats != null) { replicatorStats.msgRateIn = pubStats.msgRateIn; replicatorStats.msgThroughputIn = pubStats.msgThroughputIn; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 600b5f9475e36..034e83e67af65 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -25,8 +25,9 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.common.policies.data.ConsumerStats; -import org.apache.pulsar.common.policies.data.ReplicatorStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.util.SimpleTextOutputStream; public class NamespaceStatsAggregator { @@ -114,8 +115,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.storageReadRate = mlStats.getReadEntriesRate(); } - org.apache.pulsar.common.policies.data.TopicStats tStatus = topic.getStats(getPreciseBacklog, - subscriptionBacklogSize); + TopicStatsImpl tStatus = topic.getStats(getPreciseBacklog, subscriptionBacklogSize); stats.msgInCounter = tStatus.msgInCounter; stats.bytesInCounter = tStatus.bytesInCounter; stats.msgOutCounter = tStatus.msgOutCounter; @@ -185,7 +185,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include AggregatedSubscriptionStats subsStats = stats.subscriptionStats .computeIfAbsent(name, k -> new AggregatedSubscriptionStats()); subscription.getConsumers().forEach(consumer -> { - ConsumerStats conStats = consumer.getStats(); + ConsumerStatsImpl conStats = consumer.getStats(); AggregatedConsumerStats consumerStats = subsStats.consumerStat .computeIfAbsent(consumer, k -> new AggregatedConsumerStats()); @@ -206,7 +206,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include AggregatedReplicationStats aggReplStats = stats.replicationStats.computeIfAbsent(cluster, k -> new AggregatedReplicationStats()); - ReplicatorStats replStats = replicator.getStats(); + ReplicatorStatsImpl replStats = replicator.getStats(); aggReplStats.msgRateOut += replStats.msgRateOut; aggReplStats.msgThroughputOut += replStats.msgThroughputOut; aggReplStats.replicationBacklog += replStats.replicationBacklog; 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 78526a94f9dd8..cc23438c659fa 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 @@ -81,7 +81,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.path.PolicyPath; @@ -268,7 +268,7 @@ protected static void validateAdminAccessForTenant(PulsarService pulsar, String (isClientAuthenticated(clientAppId)), clientAppId); } - TenantInfoImpl tenantInfo = pulsar.getPulsarResources().getTenantResources().get(path(POLICIES, tenant)) + TenantInfo tenantInfo = pulsar.getPulsarResources().getTenantResources().get(path(POLICIES, tenant)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Tenant does not exist")); if (pulsar.getConfiguration().isAuthenticationEnabled() && pulsar.getConfiguration().isAuthorizationEnabled()) { @@ -322,7 +322,7 @@ protected static void validateAdminAccessForTenant(PulsarService pulsar, String } protected void validateClusterForTenant(String tenant, String cluster) { - TenantInfoImpl tenantInfo; + TenantInfo tenantInfo; try { tenantInfo = pulsar().getPulsarResources().getTenantResources().get(path(POLICIES, tenant)) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Tenant does not exist")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java index 1ece5960478b5..a62ea936ee412 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java @@ -125,7 +125,7 @@ private NamespaceBundles readBundles(NamespaceName namespace, byte[] value, long if (LOG.isDebugEnabled()) { LOG.debug("[{}] Get bundles from getLocalZkCacheService: bundles: {}, version: {}", namespace, - (localPolicies.bundles.boundaries != null) ? localPolicies.bundles : "null", + (localPolicies.bundles.getBoundaries() != null) ? localPolicies.bundles : "null", namespaceBundles.getVersion()); } return namespaceBundles; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java index f81c60d3f00c9..efb364f2d5e81 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java @@ -167,7 +167,7 @@ static long[] getPartitions(BundlesData bundlesData) { if (bundlesData == null) { return new long[]{Long.decode(FIRST_BOUNDARY), Long.decode(LAST_BOUNDARY)}; } else { - List boundaries = bundlesData.boundaries; + List boundaries = bundlesData.getBoundaries(); long[] partitions = new long[boundaries.size()]; for (int i = 0; i < boundaries.size(); i++) { partitions[i] = Long.decode(boundaries.get(i)); @@ -195,7 +195,10 @@ public BundlesData getBundlesData() { .boxed() .map(p -> format("0x%08x", p)) .collect(Collectors.toList()); - return new BundlesData(boundaries); + return BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); } public LocalPolicies toLocalPolicies() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/ConfigHelper.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/ConfigHelper.java index 108c7ac7a37bf..86cf2aad71169 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/ConfigHelper.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/ConfigHelper.java @@ -35,34 +35,34 @@ public static Map backlogQuotaMap(S } public static BacklogQuota backlogQuota(ServiceConfiguration configuration) { - return new BacklogQuota( - configuration.getBacklogQuotaDefaultLimitGB() * 1024 * 1024 * 1024, - configuration.getBacklogQuotaDefaultRetentionPolicy() - ); + return BacklogQuota.builder() + .limitSize(configuration.getBacklogQuotaDefaultLimitGB() * 1024 * 1024 * 1024) + .retentionPolicy(configuration.getBacklogQuotaDefaultRetentionPolicy()) + .build(); } public static DispatchRate topicDispatchRate(ServiceConfiguration configuration) { - return new DispatchRate( - configuration.getDispatchThrottlingRatePerTopicInMsg(), - configuration.getDispatchThrottlingRatePerTopicInByte(), - 1 - ); + return DispatchRate.builder() + .dispatchThrottlingRateInMsg(configuration.getDispatchThrottlingRatePerTopicInMsg()) + .dispatchThrottlingRateInByte(configuration.getDispatchThrottlingRatePerTopicInByte()) + .ratePeriodInSecond(1) + .build(); } public static DispatchRate subscriptionDispatchRate(ServiceConfiguration configuration) { - return new DispatchRate( - configuration.getDispatchThrottlingRatePerSubscriptionInMsg(), - configuration.getDispatchThrottlingRatePerSubscriptionInByte(), - 1 - ); + return DispatchRate.builder() + .dispatchThrottlingRateInMsg(configuration.getDispatchThrottlingRatePerSubscriptionInMsg()) + .dispatchThrottlingRateInByte(configuration.getDispatchThrottlingRatePerSubscriptionInByte()) + .ratePeriodInSecond(1) + .build(); } public static DispatchRate replicatorDispatchRate(ServiceConfiguration configuration) { - return new DispatchRate( - configuration.getDispatchThrottlingRatePerReplicatorInMsg(), - configuration.getDispatchThrottlingRatePerReplicatorInByte(), - 1 - ); + return DispatchRate.builder() + .dispatchThrottlingRateInMsg(configuration.getDispatchThrottlingRatePerReplicatorInMsg()) + .dispatchThrottlingRateInByte(configuration.getDispatchThrottlingRatePerReplicatorInByte()) + .ratePeriodInSecond(1) + .build(); } public static SubscribeRate subscribeRate(ServiceConfiguration configuration) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java index 2ee0829e16dfe..ce55717d5bb7a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java @@ -23,6 +23,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import java.net.URL; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -36,8 +37,10 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.testng.Assert; @@ -108,16 +111,16 @@ void setup() throws Exception { private void createTenant(PulsarAdmin pulsarAdmin) throws PulsarAdminException { - ClusterDataImpl clusterData = new ClusterDataImpl(); - clusterData.setServiceUrl(pulsarAdmin.getServiceUrl()); + ClusterData clusterData = ClusterData.builder() + .serviceUrl(pulsarAdmin.getServiceUrl()) + .build(); pulsarAdmins[0].clusters().createCluster("my-cluster", clusterData); Set allowedClusters = new HashSet<>(); allowedClusters.add("my-cluster"); - TenantInfoImpl adminConfig = new TenantInfoImpl(); - adminConfig.setAllowedClusters(allowedClusters); - Set adminRoles = new HashSet<>(); - adminRoles.add(""); - adminConfig.setAdminRoles(adminRoles); + TenantInfo adminConfig = TenantInfo.builder() + .adminRoles(Collections.singleton("")) + .allowedClusters(allowedClusters) + .build(); pulsarAdmin.tenants().createTenant("sla-monitor", adminConfig); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDelayedDelivery.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDelayedDelivery.java index d5658ed7a8ed1..3f010fa8eb77b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDelayedDelivery.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDelayedDelivery.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.*; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -51,7 +52,7 @@ public class AdminApiDelayedDelivery extends MockedPulsarServiceBaseTest { public void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("delayed-delivery-messages", tenantInfo); } @@ -68,7 +69,10 @@ public void testDisableDelayedDelivery() throws Exception { String namespace = "delayed-delivery-messages/default-ns"; assertNull(admin.namespaces().getDelayedDelivery(namespace)); - DelayedDeliveryPolicies delayedDeliveryPolicies = new DelayedDeliveryPolicies(2000, false); + DelayedDeliveryPolicies delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(2000) + .active(false) + .build(); admin.namespaces().setDelayedDeliveryMessages(namespace, delayedDeliveryPolicies); //zk update takes time Awaitility.await().until(() -> @@ -123,7 +127,10 @@ public void testNamespaceDelayedDeliveryPolicyApi() throws Exception { final String namespace = "delayed-delivery-messages/my-ns"; admin.namespaces().createNamespace(namespace); assertNull(admin.namespaces().getDelayedDelivery(namespace)); - DelayedDeliveryPolicies delayedDeliveryPolicies = new DelayedDeliveryPolicies(3, true); + DelayedDeliveryPolicies delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(3) + .active(true) + .build(); admin.namespaces().setDelayedDeliveryMessages(namespace, delayedDeliveryPolicies); Awaitility.await().untilAsserted(() -> assertEquals(admin.namespaces().getDelayedDelivery(namespace), delayedDeliveryPolicies)); @@ -151,12 +158,16 @@ public void testDelayedDeliveryApplied() throws Exception { assertNull(admin.topics().getDelayedDeliveryPolicy(topic)); //use broker-level by default DelayedDeliveryPolicies brokerLevelPolicy = - new DelayedDeliveryPolicies(conf.getDelayedDeliveryTickTimeMillis(), - conf.isDelayedDeliveryEnabled()); + DelayedDeliveryPolicies.builder() + .tickTime(conf.getDelayedDeliveryTickTimeMillis()) + .active(conf.isDelayedDeliveryEnabled()) + .build(); assertEquals(admin.topics().getDelayedDeliveryPolicy(topic, true), brokerLevelPolicy); //set namespace-level policy - DelayedDeliveryPolicies namespaceLevelPolicy = - new DelayedDeliveryPolicies(100, true); + DelayedDeliveryPolicies namespaceLevelPolicy = DelayedDeliveryPolicies.builder() + .tickTime(100) + .active(true) + .build(); admin.namespaces().setDelayedDeliveryMessages(namespace, namespaceLevelPolicy); Awaitility.await().untilAsserted(() -> assertNotNull(admin.namespaces().getDelayedDelivery(namespace))); @@ -164,7 +175,10 @@ public void testDelayedDeliveryApplied() throws Exception { assertEquals(policyFromBroker.getTickTime(), 100); assertTrue(policyFromBroker.isActive()); // set topic-level policy - DelayedDeliveryPolicies topicLevelPolicy = new DelayedDeliveryPolicies(200, true); + DelayedDeliveryPolicies topicLevelPolicy = DelayedDeliveryPolicies.builder() + .tickTime(200) + .active(true) + .build(); admin.topics().setDelayedDeliveryPolicy(topic, topicLevelPolicy); Awaitility.await().untilAsserted(() -> assertNotNull(admin.topics().getDelayedDeliveryPolicy(topic))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java index a0d653b200da1..b672dfd9b2eec 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java @@ -27,6 +27,7 @@ import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -70,7 +71,7 @@ public void initPersistentTopics() throws Exception { @BeforeMethod protected void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("prop", new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("prop/ns-abc"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessages.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessages.java index 7c11e04901b45..91a8f4445d1ab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessages.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessages.java @@ -29,6 +29,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; @@ -51,7 +52,7 @@ public class AdminApiMaxUnackedMessages extends MockedPulsarServiceBaseTest { public void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("max-unacked-messages", tenantInfo); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java index b28aa9cd0a01b..2894943ed64cc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java @@ -46,7 +46,9 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.OffloadedReadPriority; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -75,7 +77,7 @@ public void setup() throws Exception { super.internalSetup(); // Setup namespaces - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant(testTenant, tenantInfo); admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test")); @@ -173,15 +175,15 @@ public void testOffloadPolicies() throws Exception { long offloadDeletionLagInMillis = 100L; OffloadedReadPriority priority = OffloadedReadPriority.TIERED_STORAGE_FIRST; - OffloadPoliciesImpl offload1 = OffloadPoliciesImpl.create( + OffloadPolicies offload1 = OffloadPoliciesImpl.create( driver, region, bucket, endpoint, null, null, null, null, 100, 100, offloadThresholdInBytes, offloadDeletionLagInMillis, priority); admin.namespaces().setOffloadPolicies(namespaceName, offload1); - OffloadPoliciesImpl offload2 = (OffloadPoliciesImpl) admin.namespaces().getOffloadPolicies(namespaceName); + OffloadPolicies offload2 = admin.namespaces().getOffloadPolicies(namespaceName); assertEquals(offload1, offload2); admin.namespaces().removeOffloadPolicies(namespaceName); - OffloadPoliciesImpl offload3 = (OffloadPoliciesImpl) admin.namespaces().getOffloadPolicies(namespaceName); + OffloadPolicies offload3 = admin.namespaces().getOffloadPolicies(namespaceName); assertNull(offload3); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaAutoUpdateTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaAutoUpdateTest.java index 61387462f442e..ec2b1e8e34a72 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaAutoUpdateTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaAutoUpdateTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; @@ -55,7 +56,7 @@ public void setup() throws Exception { super.internalSetup(); // Setup namespaces - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/ns1", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java index 7153311b9470e..38d0ab7e4ec0b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.schema.StringSchema; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; @@ -66,7 +67,7 @@ public void setup() throws Exception { super.internalSetup(); // Setup namespaces - admin.clusters().createCluster(cluster, new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster(cluster, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("schematest", tenantInfo); admin.namespaces().createNamespace("schematest/test", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaValidationEnforced.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaValidationEnforced.java index 465c1dfe9772f..1ebf7f1a66c10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaValidationEnforced.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaValidationEnforced.java @@ -30,6 +30,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.protocol.schema.PostSchemaPayload; @@ -55,7 +56,7 @@ public class AdminApiSchemaValidationEnforced extends MockedPulsarServiceBaseTes public void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("schema-validation-enforced", tenantInfo); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 58255f7418284..b145e4d19a39e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -27,20 +27,20 @@ import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import static org.testng.Assert.assertThrows; import com.google.common.collect.BoundType; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Range; import com.google.common.collect.Sets; import com.google.common.hash.Hashing; - import java.lang.reflect.Field; import java.net.URL; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.List; @@ -56,7 +56,9 @@ import javax.ws.rs.client.InvocationCallback; import javax.ws.rs.client.WebTarget; import javax.ws.rs.core.Response.Status; +import lombok.Builder; import lombok.Cleanup; +import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -103,17 +105,17 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.BacklogQuota.RetentionPolicy; import org.apache.pulsar.common.policies.data.BrokerAssignment; import org.apache.pulsar.common.policies.data.BrokerInfo; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ConsumerStats; -import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; import org.apache.pulsar.common.policies.data.PartitionedTopicInternalStats; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; @@ -123,6 +125,7 @@ import org.apache.pulsar.common.policies.data.PoliciesUtil; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.Codec; @@ -138,7 +141,6 @@ import org.testng.annotations.Test; - @Slf4j @Test(groups = "broker") public class AdminApiTest extends MockedPulsarServiceBaseTest { @@ -185,7 +187,7 @@ public void setup() throws Exception { otheradmin = mockPulsarSetup.getAdmin(); // Setup namespaces - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/ns1", Sets.newHashSet("test")); @@ -233,27 +235,31 @@ public Object[][] topicNamesForAllTypesProvider() { @Test public void clusters() throws Exception { admin.clusters().createCluster("usw", - new ClusterDataImpl("http://broker.messaging.use.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.use.example.com:8080").build()); // "test" cluster is part of config-default cluster and it's znode gets created when PulsarService creates // failure-domain znode of this default cluster assertEquals(admin.clusters().getClusters(), Lists.newArrayList("test", "usw")); assertEquals(admin.clusters().getCluster("test"), - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.clusters().updateCluster("usw", - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080")); + ClusterData.builder().serviceUrl("http://new-broker.messaging.usw.example.com:8080").build()); assertEquals(admin.clusters().getClusters(), Lists.newArrayList("test", "usw")); assertEquals(admin.clusters().getCluster("usw"), - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080")); + ClusterData.builder().serviceUrl("http://new-broker.messaging.usw.example.com:8080").build()); admin.clusters().updateCluster("usw", - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080", - "https://new-broker.messaging.usw.example.com:4443")); + ClusterData.builder() + .serviceUrl("http://new-broker.messaging.usw.example.com:8080") + .serviceUrlTls("https://new-broker.messaging.usw.example.com:4443") + .build()); assertEquals(admin.clusters().getClusters(), Lists.newArrayList("test", "usw")); assertEquals(admin.clusters().getCluster("usw"), - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080", - "https://new-broker.messaging.usw.example.com:4443")); + ClusterData.builder() + .serviceUrl("http://new-broker.messaging.usw.example.com:8080") + .serviceUrlTls("https://new-broker.messaging.usw.example.com:4443") + .build()); admin.clusters().deleteCluster("usw"); Thread.sleep(300); @@ -266,7 +272,7 @@ public void clusters() throws Exception { // Check name validation try { - admin.clusters().createCluster("bf!", new ClusterDataImpl("http://dummy.messaging.example.com")); + admin.clusters().createCluster("bf!", ClusterData.builder().serviceUrl("http://dummy.messaging.example.com").build()); fail("should have failed"); } catch (PulsarAdminException e) { assertTrue(e instanceof PreconditionFailedException); @@ -278,33 +284,36 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { try { // create String policyName1 = "policy-1"; - NamespaceIsolationDataImpl nsPolicyData1 = new NamespaceIsolationDataImpl(); - nsPolicyData1.namespaces = new ArrayList(); - nsPolicyData1.namespaces.add("other/use/other.*"); - nsPolicyData1.primary = new ArrayList(); - nsPolicyData1.primary.add("prod1-broker[4-6].messaging.use.example.com"); - nsPolicyData1.secondary = new ArrayList(); - nsPolicyData1.secondary.add("prod1-broker.*.messaging.use.example.com"); - nsPolicyData1.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData1.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData1.auto_failover_policy.parameters = new HashMap(); - nsPolicyData1.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData1.auto_failover_policy.parameters.put("usage_threshold", "100"); + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData1 = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("other/use/other.*")) + .primary(Lists.newArrayList("prod1-broker[4-6].messaging.use.example.com")) + .secondary(Lists.newArrayList("prod1-broker.*.messaging.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); + admin.clusters().createNamespaceIsolationPolicy("test", policyName1, nsPolicyData1); String policyName2 = "policy-2"; - NamespaceIsolationDataImpl nsPolicyData2 = new NamespaceIsolationDataImpl(); - nsPolicyData2.namespaces = new ArrayList(); - nsPolicyData2.namespaces.add("other/use/other.*"); - nsPolicyData2.primary = new ArrayList(); - nsPolicyData2.primary.add("prod1-broker[4-6].messaging.use.example.com"); - nsPolicyData2.secondary = new ArrayList(); - nsPolicyData2.secondary.add("prod1-broker.*.messaging.use.example.com"); - nsPolicyData2.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData2.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData2.auto_failover_policy.parameters = new HashMap(); - nsPolicyData2.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData2.auto_failover_policy.parameters.put("usage_threshold", "100"); + Map parameters2 = new HashMap<>(); + parameters2.put("min_limit", "1"); + parameters2.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData2 = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("other/use/other.*")) + .primary(Collections.singletonList("prod1-broker[4-6].messaging.use.example.com")) + .secondary(Collections.singletonList("prod1-broker.*.messaging.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); admin.clusters().createNamespaceIsolationPolicy("test", policyName2, nsPolicyData2); // verify create indirectly with get @@ -313,8 +322,8 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName2), nsPolicyData2); // verify update of primary - nsPolicyData1.primary.remove(0); - nsPolicyData1.primary.add("prod1-broker[1-2].messaging.use.example.com"); + nsPolicyData1.getPrimary().remove(0); + nsPolicyData1.getPrimary().add("prod1-broker[1-2].messaging.use.example.com"); admin.clusters().updateNamespaceIsolationPolicy("test", policyName1, nsPolicyData1); // verify primary change @@ -322,8 +331,8 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName1), nsPolicyData1); // verify update of secondary - nsPolicyData1.secondary.remove(0); - nsPolicyData1.secondary.add("prod1-broker[3-4].messaging.use.example.com"); + nsPolicyData1.getSecondary().remove(0); + nsPolicyData1.getSecondary().add("prod1-broker[3-4].messaging.use.example.com"); admin.clusters().updateNamespaceIsolationPolicy("test", policyName1, nsPolicyData1); // verify secondary change @@ -331,7 +340,7 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName1), nsPolicyData1); // verify update of failover policy limit - nsPolicyData1.auto_failover_policy.parameters.put("min_limit", "10"); + nsPolicyData1.getAutoFailoverPolicy().getParameters().put("min_limit", "10"); admin.clusters().updateNamespaceIsolationPolicy("test", policyName1, nsPolicyData1); // verify min_limit change @@ -339,7 +348,7 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName1), nsPolicyData1); // verify update of failover usage_threshold limit - nsPolicyData1.auto_failover_policy.parameters.put("usage_threshold", "80"); + nsPolicyData1.getAutoFailoverPolicy().getParameters().put("usage_threshold", "80"); admin.clusters().updateNamespaceIsolationPolicy("test", policyName1, nsPolicyData1); // verify usage_threshold change @@ -419,28 +428,29 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { } // validate regex: invlid regex for primary and seconday - NamespaceIsolationDataImpl nsRegexPolicy = new NamespaceIsolationDataImpl(); - nsRegexPolicy.namespaces = new ArrayList(); - nsRegexPolicy.namespaces.add("other/use/other.*"); - nsRegexPolicy.primary = new ArrayList(); - nsRegexPolicy.primary.add("prod1-broker[45-46].messaging.use.example.com"); - nsRegexPolicy.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsRegexPolicy.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsRegexPolicy.auto_failover_policy.parameters = new HashMap(); - nsRegexPolicy.auto_failover_policy.parameters.put("min_limit", "1"); - nsRegexPolicy.auto_failover_policy.parameters.put("usage_threshold", "100"); + Map parameters = new HashMap<>(); + parameters.put("min_limit", "1"); + parameters.put("usage_threshold", "100"); + + NamespaceIsolationData.Builder nsRegexPolicy = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("other/use/other.*")) + .primary(Lists.newArrayList("prod1-broker[45-46].messaging.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build()); try { - admin.clusters().createNamespaceIsolationPolicy("test", "invalid_primary", nsRegexPolicy); + admin.clusters().createNamespaceIsolationPolicy("test", "invalid_primary", nsRegexPolicy.build()); fail("should have failed with invalid regex"); }catch (PulsarAdminException e) { //Ok } - nsRegexPolicy.primary.add("prod1-broker[4-5].messaging.use.example.com"); - nsRegexPolicy.secondary = new ArrayList(); - nsRegexPolicy.secondary.add("prod1-broker[45-46].messaging.use.example.com"); + nsRegexPolicy.primary(Lists.newArrayList("prod1-broker[45-46].messaging.use.example.com", + "prod1-broker[4-5].messaging.use.example.com")) + .secondary(Collections.singletonList("prod1-broker[45-46].messaging.use.example.com")); try { - admin.clusters().createNamespaceIsolationPolicy("test", "invalid_primary", nsRegexPolicy); + admin.clusters().createNamespaceIsolationPolicy("test", "invalid_primary", nsRegexPolicy.build()); fail("should have failed with invalid regex"); } catch (PulsarAdminException e) { // Ok @@ -683,7 +693,7 @@ public void properties() throws PulsarAdminException { @Test public void namespaces() throws Exception { - admin.clusters().createCluster("usw", new ClusterDataImpl()); + admin.clusters().createCluster("usw", ClusterData.builder().build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test", "usw")); admin.tenants().updateTenant("prop-xyz", tenantInfo); @@ -694,8 +704,8 @@ public void namespaces() throws Exception { admin.namespaces().createNamespace("prop-xyz/ns3", 4); admin.namespaces().setNamespaceReplicationClusters("prop-xyz/ns3", Sets.newHashSet("test")); - assertEquals(admin.namespaces().getPolicies("prop-xyz/ns3").bundles.numBundles, 4); - assertEquals(admin.namespaces().getPolicies("prop-xyz/ns3").bundles.boundaries.size(), 5); + assertEquals(admin.namespaces().getPolicies("prop-xyz/ns3").bundles.getNumBundles(), 4); + assertEquals(admin.namespaces().getPolicies("prop-xyz/ns3").bundles.getBoundaries().size(), 5); admin.namespaces().deleteNamespace("prop-xyz/ns3"); @@ -724,18 +734,18 @@ public void namespaces() throws Exception { Policies policies = new Policies(); policies.replication_clusters = Sets.newHashSet("test"); policies.bundles = PoliciesUtil.defaultBundle(); - policies.auth_policies.namespace_auth.put("spiffe://developer/passport-role", EnumSet.allOf(AuthAction.class)); - policies.auth_policies.namespace_auth.put("my-role", EnumSet.allOf(AuthAction.class)); + policies.auth_policies.getNamespaceAuthentication().put("spiffe://developer/passport-role", EnumSet.allOf(AuthAction.class)); + policies.auth_policies.getNamespaceAuthentication().put("my-role", EnumSet.allOf(AuthAction.class)); assertEquals(admin.namespaces().getPolicies("prop-xyz/ns1"), policies); - assertEquals(admin.namespaces().getPermissions("prop-xyz/ns1"), policies.auth_policies.namespace_auth); + assertEquals(admin.namespaces().getPermissions("prop-xyz/ns1"), policies.auth_policies.getNamespaceAuthentication()); assertEquals(admin.namespaces().getTopics("prop-xyz/ns1"), Lists.newArrayList()); admin.namespaces().revokePermissionsOnNamespace("prop-xyz/ns1", "spiffe://developer/passport-role"); admin.namespaces().revokePermissionsOnNamespace("prop-xyz/ns1", "my-role"); - policies.auth_policies.namespace_auth.remove("spiffe://developer/passport-role"); - policies.auth_policies.namespace_auth.remove("my-role"); + policies.auth_policies.getNamespaceAuthentication().remove("spiffe://developer/passport-role"); + policies.auth_policies.getNamespaceAuthentication().remove("my-role"); assertEquals(admin.namespaces().getPolicies("prop-xyz/ns1"), policies); assertEquals(admin.namespaces().getPersistence("prop-xyz/ns1"), null); @@ -813,10 +823,10 @@ public void persistentTopics(String topicName) throws Exception { publishMessagesOnPersistentTopic("persistent://prop-xyz/ns1/" + topicName, 10); TopicStats topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList(subName))); - assertEquals(topicStats.subscriptions.get(subName).consumers.size(), 1); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); - assertEquals(topicStats.publishers.size(), 0); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList(subName))); + assertEquals(topicStats.getSubscriptions().get(subName).getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); + assertEquals(topicStats.getPublishers().size(), 0); PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(persistentTopicName, false); assertEquals(internalStats.cursors.keySet(), Sets.newTreeSet(Lists.newArrayList(Codec.encode(subName)))); @@ -837,15 +847,15 @@ public void persistentTopics(String topicName) throws Exception { admin.topics().skipMessages(persistentTopicName, subName, 5); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 5); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 5); admin.topics().skipAllMessages(persistentTopicName, subName); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 0); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 0); publishNullValueMessageOnPersistentTopic(persistentTopicName, 10); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); messages = admin.topics().peekMessages(persistentTopicName, subName, 10); assertEquals(messages.size(), 10); for (int i = 0; i < 10; i++) { @@ -861,8 +871,8 @@ public void persistentTopics(String topicName) throws Exception { assertEquals(admin.topics().getSubscriptions(persistentTopicName), Lists.newArrayList()); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet()); - assertEquals(topicStats.publishers.size(), 0); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet()); + assertEquals(topicStats.getPublishers().size(), 0); try { admin.topics().skipAllMessages(persistentTopicName, subName); @@ -917,9 +927,9 @@ public void partitionedTopics(String topicType, String topicName) throws Excepti } // check the getPartitionedStats for PartitionedTopic returns only partitions metadata, and no partitions info assertEquals(admin.topics().getPartitionedTopicMetadata(partitionedTopicName).partitions, - admin.topics().getPartitionedStats(partitionedTopicName,false).metadata.partitions); + admin.topics().getPartitionedStats(partitionedTopicName,false).getMetadata().partitions); - assertEquals(admin.topics().getPartitionedStats(partitionedTopicName, false).partitions.size(), + assertEquals(admin.topics().getPartitionedStats(partitionedTopicName, false).getPartitions().size(), 0); List subscriptions = admin.topics().getSubscriptions(partitionedTopicName); @@ -943,7 +953,7 @@ public void partitionedTopics(String topicType, String topicName) throws Excepti // TODO: for non-persistent topics, deleteSubscription won't fail fail("should have failed"); } - } catch (PulsarAdminException.PreconditionFailedException e) { + } catch (PreconditionFailedException e) { // ok } catch (Exception e) { fail(e.getMessage()); @@ -985,25 +995,25 @@ public void partitionedTopics(String topicType, String topicName) throws Excepti PartitionedTopicStats topicStats = admin.topics().getPartitionedStats(partitionedTopicName, false); if (isPersistent) { // TODO: for non-persistent topics, the subscription doesn't exist - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); - assertEquals(topicStats.subscriptions.get("my-sub").consumers.size(), 1); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); + assertEquals(topicStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 10); } - assertEquals(topicStats.publishers.size(), 1); - assertEquals(topicStats.partitions, Maps.newHashMap()); + assertEquals(topicStats.getPublishers().size(), 1); + assertEquals(topicStats.getPartitions(), Maps.newHashMap()); // test per partition stats for partitioned topic topicStats = admin.topics().getPartitionedStats(partitionedTopicName, true); - assertEquals(topicStats.metadata.partitions, 4); - assertEquals(topicStats.partitions.keySet(), + assertEquals(topicStats.getMetadata().partitions, 4); + assertEquals(topicStats.getPartitions().keySet(), Sets.newHashSet(partitionedTopicName + "-partition-0", partitionedTopicName + "-partition-1", partitionedTopicName + "-partition-2", partitionedTopicName + "-partition-3")); - TopicStats partitionStats = topicStats.partitions.get(partitionedTopicName + "-partition-0"); - assertEquals(partitionStats.publishers.size(), 1); + TopicStats partitionStats = topicStats.getPartitions().get(partitionedTopicName + "-partition-0"); + assertEquals(partitionStats.getPublishers().size(), 1); if (isPersistent) { // TODO: for non-persistent topics, the subscription doesn't exist - assertEquals(partitionStats.subscriptions.get("my-sub").consumers.size(), 1); - assertEquals(partitionStats.subscriptions.get("my-sub").msgBacklog, 3, 1); + assertEquals(partitionStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get("my-sub").getMsgBacklog(), 3, 1); } try { @@ -1017,7 +1027,7 @@ public void partitionedTopics(String topicType, String topicName) throws Excepti // TODO: for non-persistent topics, skilAllMessages will cause 500 internal error admin.topics().skipAllMessages(partitionedTopicName, "my-sub"); topicStats = admin.topics().getPartitionedStats(partitionedTopicName, false); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 0); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 0); } producer.close(); @@ -1625,7 +1635,7 @@ public void testDeleteSubscription() throws Exception { try { admin.topics().deleteSubscription(persistentTopicName, subName); fail("should have failed"); - } catch (PulsarAdminException.PreconditionFailedException e) { + } catch (PreconditionFailedException e) { assertEquals(e.getStatusCode(), Status.PRECONDITION_FAILED.getStatusCode()); } @@ -1689,26 +1699,26 @@ public void testClearBacklogOnNamespace(Integer numBundles) throws Exception { admin.namespaces().clearNamespaceBacklogForSubscription("prop-xyz/ns1-bundles", "my-sub"); - long backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds2").subscriptions - .get("my-sub").msgBacklog; + long backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds2").getSubscriptions() + .get("my-sub").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds1").subscriptions - .get("my-sub").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds1").getSubscriptions() + .get("my-sub").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds1").subscriptions - .get("my-sub-1").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds1").getSubscriptions() + .get("my-sub-1").getMsgBacklog(); assertEquals(backlog, 10); admin.namespaces().clearNamespaceBacklog("prop-xyz/ns1-bundles"); - backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds1").subscriptions - .get("my-sub-1").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds1").getSubscriptions() + .get("my-sub-1").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds2").subscriptions - .get("my-sub-1").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds2").getSubscriptions() + .get("my-sub-1").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds2").subscriptions - .get("my-sub-2").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/ns1-bundles/ds2").getSubscriptions() + .get("my-sub-2").getMsgBacklog(); assertEquals(backlog, 0); } @@ -1732,7 +1742,7 @@ public void testUnsubscribeOnNamespace(Integer numBundles) throws Exception { try { admin.namespaces().unsubscribeNamespace("prop-xyz/ns1-bundles", "my-sub"); fail("should have failed"); - } catch (PulsarAdminException.PreconditionFailedException e) { + } catch (PreconditionFailedException e) { // ok } @@ -1741,7 +1751,7 @@ public void testUnsubscribeOnNamespace(Integer numBundles) throws Exception { try { admin.namespaces().unsubscribeNamespace("prop-xyz/ns1-bundles", "my-sub"); fail("should have failed"); - } catch (PulsarAdminException.PreconditionFailedException e) { + } catch (PreconditionFailedException e) { // ok } @@ -1805,11 +1815,17 @@ public void backlogQuotas() throws Exception { assertNull(quotaMap.get(BacklogQuotaType.destination_storage)); admin.namespaces().setBacklogQuota("prop-xyz/ns1", - new BacklogQuota(1 * 1024 * 1024, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(1 * 1024 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); quotaMap = admin.namespaces().getBacklogQuotaMap("prop-xyz/ns1"); assertEquals(quotaMap.size(), 1); assertEquals(quotaMap.get(BacklogQuotaType.destination_storage), - new BacklogQuota(1 * 1024 * 1024, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(1 * 1024 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); admin.namespaces().removeBacklogQuota("prop-xyz/ns1"); @@ -2200,19 +2216,25 @@ public void persistentTopicsInvalidCursorReset() throws Exception { admin.topics().delete(topicName); } + @Value + @Builder + static class CustomTenantAdmin implements TenantInfo { + private final int newTenant; + private final Set adminRoles; + private final Set allowedClusters; + } + @Test public void testObjectWithUnknownProperties() { - - class CustomTenantAdmin extends TenantInfoImpl { - @SuppressWarnings("unused") - public int newTenant; - } - - TenantInfoImpl pa = new TenantInfoImpl(Sets.newHashSet("test_appid1", "test_appid2"), Sets.newHashSet("test")); - CustomTenantAdmin cpa = new CustomTenantAdmin(); - cpa.setAdminRoles(pa.getAdminRoles()); - cpa.setAllowedClusters(pa.getAllowedClusters()); - cpa.newTenant = 100; + TenantInfo pa = TenantInfo.builder() + .adminRoles(Sets.newHashSet("test_appid1", "test_appid2")) + .allowedClusters(Sets.newHashSet("test")) + .build(); + CustomTenantAdmin cpa = CustomTenantAdmin.builder() + .adminRoles(pa.getAdminRoles()) + .allowedClusters(pa.getAllowedClusters()) + .newTenant(100) + .build(); try { admin.tenants().createTenant("test-property", cpa); @@ -2256,40 +2278,40 @@ public void testPersistentTopicsExpireMessages() throws Exception { List messageIds = publishMessagesOnPersistentTopic("persistent://prop-xyz/ns1/ds2", 10); TopicStats topicStats = admin.topics().getStats("persistent://prop-xyz/ns1/ds2"); - assertEquals(topicStats.subscriptions.get("my-sub1").msgBacklog, 10); - assertEquals(topicStats.subscriptions.get("my-sub2").msgBacklog, 10); - assertEquals(topicStats.subscriptions.get("my-sub3").msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get("my-sub1").getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get("my-sub2").getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get("my-sub3").getMsgBacklog(), 10); Thread.sleep(1000); admin.topics().expireMessages("persistent://prop-xyz/ns1/ds2", "my-sub1", 1); // Wait at most 2 seconds for sub1's message to expire. Awaitility.await().untilAsserted(() -> assertTrue( - admin.topics().getStats("persistent://prop-xyz/ns1/ds2").subscriptions.get("my-sub1").lastMarkDeleteAdvancedTimestamp > 0L)); + admin.topics().getStats("persistent://prop-xyz/ns1/ds2").getSubscriptions().get("my-sub1").getLastMarkDeleteAdvancedTimestamp() > 0L)); topicStats = admin.topics().getStats("persistent://prop-xyz/ns1/ds2"); - SubscriptionStats subStats1 = topicStats.subscriptions.get("my-sub1"); - assertEquals(subStats1.msgBacklog, 0); - SubscriptionStats subStats2 = topicStats.subscriptions.get("my-sub2"); - assertEquals(subStats2.msgBacklog, 10); - assertEquals(subStats2.lastMarkDeleteAdvancedTimestamp, 0L); - SubscriptionStats subStats3 = topicStats.subscriptions.get("my-sub3"); - assertEquals(subStats3.msgBacklog, 10); - assertEquals(subStats3.lastMarkDeleteAdvancedTimestamp, 0L); + SubscriptionStats subStats1 = topicStats.getSubscriptions().get("my-sub1"); + assertEquals(subStats1.getMsgBacklog(), 0); + SubscriptionStats subStats2 = topicStats.getSubscriptions().get("my-sub2"); + assertEquals(subStats2.getMsgBacklog(), 10); + assertEquals(subStats2.getLastMarkDeleteAdvancedTimestamp(), 0L); + SubscriptionStats subStats3 = topicStats.getSubscriptions().get("my-sub3"); + assertEquals(subStats3.getMsgBacklog(), 10); + assertEquals(subStats3.getLastMarkDeleteAdvancedTimestamp(), 0L); admin.topics().expireMessages("persistent://prop-xyz/ns1/ds2", "my-sub2", messageIds.get(4), false); // Wait at most 2 seconds for sub2's message to expire. Awaitility.await().untilAsserted(() -> assertTrue( - admin.topics().getStats("persistent://prop-xyz/ns1/ds2").subscriptions.get("my-sub2").lastMarkDeleteAdvancedTimestamp > 0L)); + admin.topics().getStats("persistent://prop-xyz/ns1/ds2").getSubscriptions().get("my-sub2").getLastMarkDeleteAdvancedTimestamp() > 0L)); topicStats = admin.topics().getStats("persistent://prop-xyz/ns1/ds2"); - subStats1 = topicStats.subscriptions.get("my-sub1"); - assertEquals(subStats1.msgBacklog, 0); - assertTrue(subStats1.lastMarkDeleteAdvancedTimestamp > 0L); - long sub2lastMarkDeleteAdvancedTimestamp = subStats1.lastMarkDeleteAdvancedTimestamp; - subStats2 = topicStats.subscriptions.get("my-sub2"); - assertEquals(subStats2.msgBacklog, 5); - subStats3 = topicStats.subscriptions.get("my-sub3"); - assertEquals(subStats3.msgBacklog, 10); - assertEquals(subStats3.lastMarkDeleteAdvancedTimestamp, 0L); + subStats1 = topicStats.getSubscriptions().get("my-sub1"); + assertEquals(subStats1.getMsgBacklog(), 0); + assertTrue(subStats1.getLastMarkDeleteAdvancedTimestamp() > 0L); + long sub2lastMarkDeleteAdvancedTimestamp = subStats1.getLastMarkDeleteAdvancedTimestamp(); + subStats2 = topicStats.getSubscriptions().get("my-sub2"); + assertEquals(subStats2.getMsgBacklog(), 5); + subStats3 = topicStats.getSubscriptions().get("my-sub3"); + assertEquals(subStats3.getMsgBacklog(), 10); + assertEquals(subStats3.getLastMarkDeleteAdvancedTimestamp(), 0L); try { admin.topics().expireMessagesForAllSubscriptions("persistent://prop-xyz/ns1/ds2", 1); @@ -2299,17 +2321,17 @@ public void testPersistentTopicsExpireMessages() throws Exception { } // Wait at most 2 seconds for sub3's message to expire. Awaitility.await().untilAsserted(() -> assertTrue( - admin.topics().getStats("persistent://prop-xyz/ns1/ds2").subscriptions.get("my-sub3").lastMarkDeleteAdvancedTimestamp > 0L)); + admin.topics().getStats("persistent://prop-xyz/ns1/ds2").getSubscriptions().get("my-sub3").getLastMarkDeleteAdvancedTimestamp() > 0L)); topicStats = admin.topics().getStats("persistent://prop-xyz/ns1/ds2"); - subStats1 = topicStats.subscriptions.get("my-sub1"); - assertEquals(subStats1.msgBacklog, 0); - assertEquals(subStats1.lastMarkDeleteAdvancedTimestamp, subStats1.lastMarkDeleteAdvancedTimestamp); + subStats1 = topicStats.getSubscriptions().get("my-sub1"); + assertEquals(subStats1.getMsgBacklog(), 0); + assertEquals(subStats1.getLastMarkDeleteAdvancedTimestamp(), subStats1.getLastMarkDeleteAdvancedTimestamp()); // Wait at most 2 seconds for rest of sub2's message to expire. - subStats2 = topicStats.subscriptions.get("my-sub2"); - assertEquals(subStats2.msgBacklog, 0); - assertTrue(subStats2.lastMarkDeleteAdvancedTimestamp > sub2lastMarkDeleteAdvancedTimestamp); - subStats3 = topicStats.subscriptions.get("my-sub3"); - assertEquals(subStats3.msgBacklog, 0); + subStats2 = topicStats.getSubscriptions().get("my-sub2"); + assertEquals(subStats2.getMsgBacklog(), 0); + assertTrue(subStats2.getLastMarkDeleteAdvancedTimestamp() > sub2lastMarkDeleteAdvancedTimestamp); + subStats3 = topicStats.getSubscriptions().get("my-sub3"); + assertEquals(subStats3.getMsgBacklog(), 0); consumer1.close(); consumer2.close(); @@ -2375,24 +2397,24 @@ public void testPersistentTopicExpireMessageOnPartitionTopic() throws Exception PartitionedTopicStats topicStats = admin.topics().getPartitionedStats("persistent://prop-xyz/ns1/ds1", true); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 10); - TopicStats partitionStatsPartition0 = topicStats.partitions + TopicStats partitionStatsPartition0 = topicStats.getPartitions() .get("persistent://prop-xyz/ns1/ds1-partition-0"); - TopicStats partitionStatsPartition1 = topicStats.partitions + TopicStats partitionStatsPartition1 = topicStats.getPartitions() .get("persistent://prop-xyz/ns1/ds1-partition-1"); - assertEquals(partitionStatsPartition0.subscriptions.get("my-sub").msgBacklog, 3, 1); - assertEquals(partitionStatsPartition1.subscriptions.get("my-sub").msgBacklog, 3, 1); + assertEquals(partitionStatsPartition0.getSubscriptions().get("my-sub").getMsgBacklog(), 3, 1); + assertEquals(partitionStatsPartition1.getSubscriptions().get("my-sub").getMsgBacklog(), 3, 1); Thread.sleep(1000); admin.topics().expireMessagesForAllSubscriptions("persistent://prop-xyz/ns1/ds1", 1); Thread.sleep(1000); topicStats = admin.topics().getPartitionedStats("persistent://prop-xyz/ns1/ds1", true); - partitionStatsPartition0 = topicStats.partitions.get("persistent://prop-xyz/ns1/ds1-partition-0"); - partitionStatsPartition1 = topicStats.partitions.get("persistent://prop-xyz/ns1/ds1-partition-1"); - assertEquals(partitionStatsPartition0.subscriptions.get("my-sub").msgBacklog, 0); - assertEquals(partitionStatsPartition1.subscriptions.get("my-sub").msgBacklog, 0); + partitionStatsPartition0 = topicStats.getPartitions().get("persistent://prop-xyz/ns1/ds1-partition-0"); + partitionStatsPartition1 = topicStats.getPartitions().get("persistent://prop-xyz/ns1/ds1-partition-1"); + assertEquals(partitionStatsPartition0.getSubscriptions().get("my-sub").getMsgBacklog(), 0); + assertEquals(partitionStatsPartition1.getSubscriptions().get("my-sub").getMsgBacklog(), 0); producer.close(); consumer.close(); @@ -2560,8 +2582,8 @@ public void failed(Throwable e) { uriStats.completeExceptionally(e); } }); - assertEquals(urlStats.get().subscriptions.size(), 1); - assertEquals(uriStats.get().subscriptions.size(), 1); + assertEquals(urlStats.get().getSubscriptions().size(), 1); + assertEquals(uriStats.get().getSubscriptions().size(), 1); } static class MockedPulsarService extends MockedPulsarServiceBaseTest { @@ -2595,12 +2617,12 @@ public PulsarAdmin getAdmin() { @Test public void testTopicBundleRangeLookup() throws PulsarAdminException, PulsarServerException, Exception { - admin.clusters().createCluster("usw", new ClusterDataImpl()); + admin.clusters().createCluster("usw", ClusterData.builder().build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test", "usw")); admin.tenants().updateTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/getBundleNs", 100); - assertEquals(admin.namespaces().getPolicies("prop-xyz/getBundleNs").bundles.numBundles, 100); + assertEquals(admin.namespaces().getPolicies("prop-xyz/getBundleNs").bundles.getNumBundles(), 100); // (1) create a topic final String topicName = "persistent://prop-xyz/getBundleNs/topic1"; @@ -2736,12 +2758,12 @@ public void testTopicStatsLastExpireTimestampForSubscription() throws PulsarAdmi .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); - Assert.assertEquals(admin.topics().getStats(topic).subscriptions.size(), 1); - Assert.assertEquals(admin.topics().getStats(topic).subscriptions.values().iterator().next().lastExpireTimestamp, 0L); + Assert.assertEquals(admin.topics().getStats(topic).getSubscriptions().size(), 1); + Assert.assertEquals(admin.topics().getStats(topic).getSubscriptions().values().iterator().next().getLastExpireTimestamp(), 0L); Thread.sleep(10000); // Update policy to trigger message expiry check. admin.namespaces().setNamespaceMessageTTL("prop-xyz/ns1", 5); - Awaitility.await().until(() -> admin.topics().getStats(topic).subscriptions.values().iterator().next().lastExpireTimestamp > 0L); + Awaitility.await().until(() -> admin.topics().getStats(topic).getSubscriptions().values().iterator().next().getLastExpireTimestamp() > 0L); } @Test(timeOut = 150000) @@ -2779,7 +2801,7 @@ public void testSubscriptionExpiry() throws Exception { @Test public void testCreateAndDeleteNamespaceWithBundles() throws Exception { - admin.clusters().createCluster("usw", new ClusterDataImpl()); + admin.clusters().createCluster("usw", ClusterData.builder().build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test", "usw")); admin.tenants().updateTenant("prop-xyz", tenantInfo); @@ -2811,7 +2833,7 @@ public void testBacklogSizeShouldBeZeroWhenConsumerAckedAllMessages() throws Exc } TopicStats topicStats = admin.topics().getStats(topic); - assertEquals(topicStats.subscriptions.get("sub-1").lastMarkDeleteAdvancedTimestamp, 0L); + assertEquals(topicStats.getSubscriptions().get("sub-1").getLastMarkDeleteAdvancedTimestamp(), 0L); for (int i = 0; i < messages; i++) { consumer.acknowledgeCumulative(consumer.receive()); @@ -2821,8 +2843,8 @@ public void testBacklogSizeShouldBeZeroWhenConsumerAckedAllMessages() throws Exc Thread.sleep(1000); topicStats = admin.topics().getStats(topic); - assertEquals(topicStats.backlogSize, 0); - assertTrue(topicStats.subscriptions.get("sub-1").lastMarkDeleteAdvancedTimestamp > 0L); + assertEquals(topicStats.getBacklogSize(), 0); + assertTrue(topicStats.getSubscriptions().get("sub-1").getLastMarkDeleteAdvancedTimestamp() > 0L); } @Test @@ -2856,11 +2878,11 @@ public void testGetReadPositionWhenJoining() throws Exception { } TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.subscriptions.size(), 1); - SubscriptionStats subStats = stats.subscriptions.get(subName); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + SubscriptionStats subStats = stats.getSubscriptions().get(subName); Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.consumers.size(), 2); - ConsumerStats consumerStats = subStats.consumers.get(0); + Assert.assertEquals(subStats.getConsumers().size(), 2); + ConsumerStats consumerStats = subStats.getConsumers().get(0); Assert.assertEquals(consumerStats.getReadPositionWhenJoining(), PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId() + 1).toString()); } @@ -2895,33 +2917,33 @@ public void testPartitionedTopicMsgDelayedAggregated() throws Exception { } PartitionedTopicStats partitionedTopicStats = admin.topics().getPartitionedStats(topic, false); Assert.assertNotNull(partitionedTopicStats); - SubscriptionStats subStats = partitionedTopicStats.subscriptions.get(subName); + SubscriptionStats subStats = partitionedTopicStats.getSubscriptions().get(subName); Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.msgBacklog, subStats.msgBacklogNoDelayed + subStats.msgDelayed); + Assert.assertEquals(subStats.getMsgBacklog(), subStats.getMsgBacklogNoDelayed() + subStats.getMsgDelayed()); partitionedTopicStats = admin.topics().getPartitionedStats(topic, true); Assert.assertNotNull(partitionedTopicStats); - subStats = partitionedTopicStats.subscriptions.get(subName); + subStats = partitionedTopicStats.getSubscriptions().get(subName); Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.msgBacklog, subStats.msgBacklogNoDelayed + subStats.msgDelayed); - Assert.assertNotNull(partitionedTopicStats.partitions); - Assert.assertEquals(partitionedTopicStats.partitions.size(), numPartitions); + Assert.assertEquals(subStats.getMsgBacklog(), subStats.getMsgBacklogNoDelayed() + subStats.getMsgDelayed()); + Assert.assertNotNull(partitionedTopicStats.getPartitions()); + Assert.assertEquals(partitionedTopicStats.getPartitions().size(), numPartitions); long sumMsgBacklog = 0; long sumMsgBacklogNoDelayed = 0; long sumMsgDelayed = 0; - for(TopicStats stats: partitionedTopicStats.partitions.values()){ + for(TopicStats stats: partitionedTopicStats.getPartitions().values()){ Assert.assertNotNull(stats); - SubscriptionStats partitionedSubStats = stats.subscriptions.get(subName); + SubscriptionStats partitionedSubStats = stats.getSubscriptions().get(subName); Assert.assertNotNull(partitionedSubStats); - sumMsgBacklog += partitionedSubStats.msgBacklog; - sumMsgBacklogNoDelayed += partitionedSubStats.msgBacklogNoDelayed; - sumMsgDelayed += partitionedSubStats.msgDelayed; + sumMsgBacklog += partitionedSubStats.getMsgBacklog(); + sumMsgBacklogNoDelayed += partitionedSubStats.getMsgBacklogNoDelayed(); + sumMsgDelayed += partitionedSubStats.getMsgDelayed(); } Assert.assertEquals(sumMsgBacklog, sumMsgBacklogNoDelayed + sumMsgDelayed); - Assert.assertEquals(sumMsgBacklog, subStats.msgBacklog); - Assert.assertEquals(sumMsgBacklogNoDelayed, subStats.msgBacklogNoDelayed); - Assert.assertEquals(sumMsgDelayed, subStats.msgDelayed); + Assert.assertEquals(sumMsgBacklog, subStats.getMsgBacklog()); + Assert.assertEquals(sumMsgBacklogNoDelayed, subStats.getMsgBacklogNoDelayed()); + Assert.assertEquals(sumMsgDelayed, subStats.getMsgDelayed()); } @Test(timeOut = 20000) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java index d9e1eaee2f641..3ca5ea96ed2ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin; import static org.apache.commons.lang3.StringUtils.isBlank; +import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -78,14 +79,17 @@ 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.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationData; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.FailureDomain; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistencePolicies; @@ -120,7 +124,7 @@ public void setup() throws Exception { mockPulsarSetup.setup(); // Setup namespaces - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/ns1", Sets.newHashSet("test")); @@ -223,26 +227,26 @@ public void testIncrementPartitionsOfTopic() throws Exception { // test cumulative stats for partitioned topic PartitionedTopicStats topicStats = admin.topics().getPartitionedStats(partitionedTopicName, false); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList(subName1, subName2))); - assertEquals(topicStats.subscriptions.get(subName2).consumers.size(), 1); - assertEquals(topicStats.subscriptions.get(subName2).msgBacklog, totalMessages); - assertEquals(topicStats.publishers.size(), 1); - assertEquals(topicStats.partitions, Maps.newHashMap()); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList(subName1, subName2))); + assertEquals(topicStats.getSubscriptions().get(subName2).getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get(subName2).getMsgBacklog(), totalMessages); + assertEquals(topicStats.getPublishers().size(), 1); + assertEquals(topicStats.getPartitions(), Maps.newHashMap()); // (5) verify: each partition should have backlog topicStats = admin.topics().getPartitionedStats(partitionedTopicName, true); - assertEquals(topicStats.metadata.partitions, newPartitions); + assertEquals(topicStats.getMetadata().partitions, newPartitions); Set partitionSet = Sets.newHashSet(); for (int i = 0; i < newPartitions; i++) { partitionSet.add(partitionedTopicName + "-partition-" + i); } - assertEquals(topicStats.partitions.keySet(), partitionSet); + assertEquals(topicStats.getPartitions().keySet(), partitionSet); for (int i = 0; i < newPartitions; i++) { - TopicStats partitionStats = topicStats.partitions + TopicStats partitionStats = topicStats.getPartitions() .get(TopicName.get(partitionedTopicName).getPartition(i).toString()); - assertEquals(partitionStats.publishers.size(), 1); - assertEquals(partitionStats.subscriptions.get(subName2).consumers.size(), 1); - assertEquals(partitionStats.subscriptions.get(subName2).msgBacklog, 2, 1); + assertEquals(partitionStats.getPublishers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get(subName2).getConsumers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get(subName2).getMsgBacklog(), 2, 1); } producer.close(); @@ -258,8 +262,8 @@ public void testTopicPoliciesWithMultiBroker() throws Exception { conf.setSystemTopicEnabled(true); conf.setTopicLevelPoliciesEnabled(true); super.internalSetup(); - admin.clusters().createCluster("test" - , new ClusterDataImpl(pulsar.getWebServiceAddress() + ",localhost:1026," + "localhost:2050")); + admin.clusters().createCluster("test", + ClusterData.builder().serviceUrl((pulsar.getWebServiceAddress() + ",localhost:1026," + "localhost:2050")).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/ns1", Sets.newHashSet("test")); @@ -340,9 +344,9 @@ public void nonPersistentTopics() throws Exception { publishMessagesOnTopic("non-persistent://prop-xyz/ns1/" + topicName, 10, 0); TopicStats topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); - assertEquals(topicStats.subscriptions.get("my-sub").consumers.size(), 1); - assertEquals(topicStats.publishers.size(), 0); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); + assertEquals(topicStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); + assertEquals(topicStats.getPublishers().size(), 0); PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(persistentTopicName, false); assertEquals(internalStats.cursors.keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); @@ -350,8 +354,8 @@ public void nonPersistentTopics() throws Exception { consumer.close(); topicStats = admin.topics().getStats(persistentTopicName); - assertTrue(topicStats.subscriptions.containsKey("my-sub")); - assertEquals(topicStats.publishers.size(), 0); + assertTrue(topicStats.getSubscriptions().containsKey("my-sub")); + assertEquals(topicStats.getPublishers().size(), 0); // test partitioned-topic final String partitionedTopicName = "non-persistent://prop-xyz/ns1/paritioned"; @@ -655,13 +659,13 @@ public void testLoadReportApi() throws Exception { @Test public void testPeerCluster() throws Exception { admin.clusters().createCluster("us-west1", - new ClusterDataImpl("http://broker.messaging.west1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west1.example.com:8080").build()); admin.clusters().createCluster("us-west2", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-east1", - new ClusterDataImpl("http://broker.messaging.east1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.east1.example.com:8080").build()); admin.clusters().createCluster("us-east2", - new ClusterDataImpl("http://broker.messaging.east2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.east2.example.com:8080").build()); admin.clusters().updatePeerClusterNames("us-west1", Sets.newLinkedHashSet(Lists.newArrayList("us-west2"))); assertEquals(admin.clusters().getCluster("us-west1").getPeerClusterNames(), Lists.newArrayList("us-west2")); @@ -700,18 +704,18 @@ public void testPeerCluster() throws Exception { @Test public void testReplicationPeerCluster() throws Exception { admin.clusters().createCluster("us-west1", - new ClusterDataImpl("http://broker.messaging.west1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west1.example.com:8080").build()); admin.clusters().createCluster("us-west2", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-west3", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-west4", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-east1", - new ClusterDataImpl("http://broker.messaging.east1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.east1.example.com:8080").build()); admin.clusters().createCluster("us-east2", - new ClusterDataImpl("http://broker.messaging.east2.example.com:8080")); - admin.clusters().createCluster("global", new ClusterDataImpl()); + ClusterData.builder().serviceUrl("http://broker.messaging.east2.example.com:8080").build()); + admin.clusters().createCluster("global", ClusterData.builder().build()); List allClusters = admin.clusters().getClusters(); Collections.sort(allClusters); @@ -759,8 +763,9 @@ public void clusterFailureDomain() throws PulsarAdminException { final String cluster = pulsar.getConfiguration().getClusterName(); // create - FailureDomainImpl domain = new FailureDomainImpl(); - domain.setBrokers(Sets.newHashSet("b1", "b2", "b3")); + FailureDomain domain = FailureDomain.builder() + .brokers(Sets.newHashSet("b1", "b2", "b3")) + .build(); admin.clusters().createFailureDomain(cluster, "domain-1", domain); admin.clusters().updateFailureDomain(cluster, "domain-1", domain); @@ -866,25 +871,25 @@ public void testPublishConsumerStats() throws Exception { TopicStats stats; try { stats = admin.topics().getStats(topic); - return stats.publishers.size() > 0 && stats.subscriptions.get(subscriberName) != null - && stats.subscriptions.get(subscriberName).consumers.size() > 0; + return stats.getPublishers().size() > 0 && stats.getSubscriptions().get(subscriberName) != null + && stats.getSubscriptions().get(subscriberName).getConsumers().size() > 0; } catch (PulsarAdminException e) { return false; } }, 5, 200); TopicStats topicStats = admin.topics().getStats(topic); - assertEquals(topicStats.publishers.size(), 1); - assertNotNull(topicStats.publishers.get(0).getAddress()); - assertNotNull(topicStats.publishers.get(0).getClientVersion()); - assertNotNull(topicStats.publishers.get(0).getConnectedSince()); - assertNotNull(topicStats.publishers.get(0).getProducerName()); - assertEquals(topicStats.publishers.get(0).getProducerName(), producerName); - - SubscriptionStats subscriber = topicStats.subscriptions.get(subscriberName); + assertEquals(topicStats.getPublishers().size(), 1); + assertNotNull(topicStats.getPublishers().get(0).getAddress()); + assertNotNull(topicStats.getPublishers().get(0).getClientVersion()); + assertNotNull(topicStats.getPublishers().get(0).getConnectedSince()); + assertNotNull(topicStats.getPublishers().get(0).getProducerName()); + assertEquals(topicStats.getPublishers().get(0).getProducerName(), producerName); + + SubscriptionStats subscriber = topicStats.getSubscriptions().get(subscriberName); assertNotNull(subscriber); - assertEquals(subscriber.consumers.size(), 1); - ConsumerStats consumerStats = subscriber.consumers.get(0); + assertEquals(subscriber.getConsumers().size(), 1); + ConsumerStats consumerStats = subscriber.getConsumers().get(0); assertNotNull(consumerStats.getAddress()); assertNotNull(consumerStats.getClientVersion()); assertNotNull(consumerStats.getConnectedSince()); @@ -906,7 +911,7 @@ public void testTenantNameWithUnderscore() throws Exception { .create(); TopicStats stats = admin.topics().getStats(topic); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); producer.close(); } @@ -969,18 +974,20 @@ public void brokerNamespaceIsolationPolicies() throws Exception { String namespaceRegex = "other/" + cluster + "/other.*"; String brokerName = pulsar.getAdvertisedAddress(); String brokerAddress = brokerName + ":" + pulsar.getConfiguration().getWebServicePort().get(); - NamespaceIsolationDataImpl nsPolicyData1 = new NamespaceIsolationDataImpl(); - nsPolicyData1.namespaces = new ArrayList<>(); - nsPolicyData1.namespaces.add(namespaceRegex); - nsPolicyData1.primary = new ArrayList<>(); - nsPolicyData1.primary.add(brokerName + ":[0-9]*"); - nsPolicyData1.secondary = new ArrayList<>(); - nsPolicyData1.secondary.add(brokerName + ".*"); - nsPolicyData1.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData1.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData1.auto_failover_policy.parameters = new HashMap<>(); - nsPolicyData1.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData1.auto_failover_policy.parameters.put("usage_threshold", "100"); + + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData1 = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList(namespaceRegex)) + .primary(Collections.singletonList(brokerName + ":[0-9]*")) + .secondary(Collections.singletonList(brokerName + ".*")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); admin.clusters().createNamespaceIsolationPolicy(cluster, policyName1, nsPolicyData1); List brokerIsolationDataList = admin.clusters() @@ -992,13 +999,13 @@ public void brokerNamespaceIsolationPolicies() throws Exception { BrokerNamespaceIsolationDataImpl brokerIsolationData = (BrokerNamespaceIsolationDataImpl) admin.clusters() .getBrokerWithNamespaceIsolationPolicy(cluster, brokerAddress); - assertEquals(brokerIsolationData.brokerName, brokerAddress); - assertEquals(brokerIsolationData.namespaceRegex.size(), 1); - assertEquals(brokerIsolationData.namespaceRegex.get(0), namespaceRegex); + assertEquals(brokerIsolationData.getBrokerName(), brokerAddress); + assertEquals(brokerIsolationData.getNamespaceRegex().size(), 1); + assertEquals(brokerIsolationData.getNamespaceRegex().get(0), namespaceRegex); BrokerNamespaceIsolationDataImpl isolationData = (BrokerNamespaceIsolationDataImpl) admin.clusters() .getBrokerWithNamespaceIsolationPolicy(cluster, "invalid-broker"); - assertFalse(isolationData.isPrimary); + assertFalse(isolationData.isPrimary()); } // create 1 namespace: @@ -1019,16 +1026,19 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { // create String policyName1 = "policy-1"; String cluster = pulsar.getConfiguration().getClusterName(); - NamespaceIsolationDataImpl nsPolicyData1 = new NamespaceIsolationDataImpl(); - nsPolicyData1.namespaces = new ArrayList<>(); - nsPolicyData1.namespaces.add(ns1Name); - nsPolicyData1.primary = new ArrayList<>(); - nsPolicyData1.primary.add(brokerName + ".*"); - nsPolicyData1.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData1.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData1.auto_failover_policy.parameters = new HashMap<>(); - nsPolicyData1.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData1.auto_failover_policy.parameters.put("usage_threshold", "100"); + + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData1 = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList(ns1Name)) + .primary(Collections.singletonList(brokerName + ".*")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); admin.clusters().createNamespaceIsolationPolicyAsync(cluster, policyName1, nsPolicyData1).get(); // 1. with matched isolation broker configured and matched, lookup will success. @@ -1037,8 +1047,8 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { log.info(" 1 get lookup url {}", brokerUrl); // 2. update isolation policy, without broker matched, lookup will fail. - nsPolicyData1.primary = new ArrayList<>(); - nsPolicyData1.primary.add(brokerName + "not_match"); + nsPolicyData1.getPrimary().clear();; + nsPolicyData1.getPrimary().add(brokerName + "not_match"); admin.clusters().updateNamespaceIsolationPolicyAsync(cluster, policyName1, nsPolicyData1).get(); try { @@ -1059,7 +1069,8 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { @Test public void clustersList() throws PulsarAdminException { final String cluster = pulsar.getConfiguration().getClusterName(); - admin.clusters().createCluster("global", new ClusterDataImpl("http://localhost:6650")); + admin.clusters().createCluster("global", ClusterData.builder() + .serviceUrl("http://localhost:6650").build()); // Global cluster, if there, should be omitted from the results assertEquals(admin.clusters().getClusters(), Lists.newArrayList(cluster)); @@ -1134,12 +1145,12 @@ public void testConsumerStatsLastTimestamp() throws PulsarClientException, Pulsa // Get the consumer stats. TopicStats topicStats = admin.topics().getStats(topic); - SubscriptionStats subscriptionStats = topicStats.subscriptions.get(subscribeName); - long startConsumedFlowTimestamp = subscriptionStats.lastConsumedFlowTimestamp; - long startAckedTimestampInSubStats = subscriptionStats.lastAckedTimestamp; - ConsumerStats consumerStats = subscriptionStats.consumers.get(0); - long startConsumedTimestampInConsumerStats = consumerStats.lastConsumedTimestamp; - long startAckedTimestampInConsumerStats = consumerStats.lastAckedTimestamp; + SubscriptionStats subscriptionStats = topicStats.getSubscriptions().get(subscribeName); + long startConsumedFlowTimestamp = subscriptionStats.getLastConsumedFlowTimestamp(); + long startAckedTimestampInSubStats = subscriptionStats.getLastAckedTimestamp(); + ConsumerStats consumerStats = subscriptionStats.getConsumers().get(0); + long startConsumedTimestampInConsumerStats = consumerStats.getLastConsumedTimestamp(); + long startAckedTimestampInConsumerStats = consumerStats.getLastAckedTimestamp(); // Because the message was pushed by the broker, the consumedTimestamp should not as 0. assertNotEquals(0, startConsumedTimestampInConsumerStats); @@ -1153,7 +1164,7 @@ public void testConsumerStatsLastTimestamp() throws PulsarClientException, Pulsa // Waiting for the ack command send to the broker. while (true) { topicStats = admin.topics().getStats(topic); - if (topicStats.subscriptions.get(subscribeName).lastAckedTimestamp != 0) { + if (topicStats.getSubscriptions().get(subscribeName).getLastAckedTimestamp() != 0) { break; } TimeUnit.MILLISECONDS.sleep(100); @@ -1161,12 +1172,12 @@ public void testConsumerStatsLastTimestamp() throws PulsarClientException, Pulsa // Get the consumer stats. topicStats = admin.topics().getStats(topic); - subscriptionStats = topicStats.subscriptions.get(subscribeName); - long consumedFlowTimestamp = subscriptionStats.lastConsumedFlowTimestamp; - long ackedTimestampInSubStats = subscriptionStats.lastAckedTimestamp; - consumerStats = subscriptionStats.consumers.get(0); - long consumedTimestamp = consumerStats.lastConsumedTimestamp; - long ackedTimestamp = consumerStats.lastAckedTimestamp; + subscriptionStats = topicStats.getSubscriptions().get(subscribeName); + long consumedFlowTimestamp = subscriptionStats.getLastConsumedFlowTimestamp(); + long ackedTimestampInSubStats = subscriptionStats.getLastAckedTimestamp(); + consumerStats = subscriptionStats.getConsumers().get(0); + long consumedTimestamp = consumerStats.getLastConsumedTimestamp(); + long ackedTimestamp = consumerStats.getLastAckedTimestamp(); // The lastConsumedTimestamp should same as the last time because the broker does not push any messages and the // consumer does not pull any messages. @@ -1184,7 +1195,7 @@ public void testConsumerStatsLastTimestamp() throws PulsarClientException, Pulsa // Waiting for the ack command send to the broker. while (true) { topicStats = admin.topics().getStats(topic); - if (topicStats.subscriptions.get(subscribeName).lastAckedTimestamp != ackedTimestampInSubStats) { + if (topicStats.getSubscriptions().get(subscribeName).getLastAckedTimestamp() != ackedTimestampInSubStats) { break; } TimeUnit.MILLISECONDS.sleep(100); @@ -1192,13 +1203,13 @@ public void testConsumerStatsLastTimestamp() throws PulsarClientException, Pulsa // Get the consumer stats again. topicStats = admin.topics().getStats(topic); - subscriptionStats = topicStats.subscriptions.get(subscribeName); - long lastConsumedFlowTimestamp = subscriptionStats.lastConsumedFlowTimestamp; - long lastConsumedTimestampInSubStats = subscriptionStats.lastConsumedTimestamp; - long lastAckedTimestampInSubStats = subscriptionStats.lastAckedTimestamp; - consumerStats = subscriptionStats.consumers.get(0); - long lastConsumedTimestamp = consumerStats.lastConsumedTimestamp; - long lastAckedTimestamp = consumerStats.lastAckedTimestamp; + subscriptionStats = topicStats.getSubscriptions().get(subscribeName); + long lastConsumedFlowTimestamp = subscriptionStats.getLastConsumedFlowTimestamp(); + long lastConsumedTimestampInSubStats = subscriptionStats.getLastConsumedTimestamp(); + long lastAckedTimestampInSubStats = subscriptionStats.getLastAckedTimestamp(); + consumerStats = subscriptionStats.getConsumers().get(0); + long lastConsumedTimestamp = consumerStats.getLastConsumedTimestamp(); + long lastAckedTimestamp = consumerStats.getLastAckedTimestamp(); assertTrue(consumedTimestamp < lastConsumedTimestamp); assertTrue(ackedTimestamp < lastAckedTimestamp); @@ -1242,11 +1253,11 @@ public void testPreciseBacklog() throws PulsarClientException, PulsarAdminExcept assertNotNull(subscription); ((ManagedLedgerImpl)subscription.getCursor().getManagedLedger()).setEntriesAddedCounter(10L); TopicStats topicStats = admin.topics().getStats(topic); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); topicStats = admin.topics().getStats(topic, true, true); - assertEquals(topicStats.subscriptions.get(subName).backlogSize, 43); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 1); + assertEquals(topicStats.getSubscriptions().get(subName).getBacklogSize(), 43); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 1); consumer.acknowledge(message); // wait for ack send @@ -1254,11 +1265,11 @@ public void testPreciseBacklog() throws PulsarClientException, PulsarAdminExcept // Consumer acks the message, so the precise backlog is 0 topicStats = admin.topics().getStats(topic, true, true); - assertEquals(topicStats.subscriptions.get(subName).backlogSize, 0); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 0); + assertEquals(topicStats.getSubscriptions().get(subName).getBacklogSize(), 0); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 0); topicStats = admin.topics().getStats(topic); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 9); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 9); } @Test(timeOut = 30000) @@ -1300,8 +1311,8 @@ public void testBacklogNoDelayed() throws PulsarClientException, PulsarAdminExce Thread.sleep(500); TopicStats topicStats = admin.topics().getStats(topic, true, true); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); - assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 5); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklogNoDelayed(), 5); for (int i = 0; i < 5; i++) { consumer.acknowledge(consumer.receive()); @@ -1309,8 +1320,8 @@ public void testBacklogNoDelayed() throws PulsarClientException, PulsarAdminExce // Wait the ack send. Thread.sleep(500); topicStats = admin.topics().getStats(topic, true, true); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 5); - assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 0); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 5); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklogNoDelayed(), 0); } @Test @@ -1348,11 +1359,11 @@ public void testPreciseBacklogForPartitionedTopic() throws PulsarClientException } TopicStats topicStats = admin.topics().getPartitionedStats(topic, false); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 20); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 20); topicStats = admin.topics().getPartitionedStats(topic, false, true, true); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 1); - assertEquals(topicStats.subscriptions.get(subName).backlogSize, 43); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 1); + assertEquals(topicStats.getSubscriptions().get(subName).getBacklogSize(), 43); } @Test(timeOut = 30000) @@ -1390,9 +1401,9 @@ public void testBacklogNoDelayedForPartitionedTopic() throws PulsarClientExcepti } TopicStats topicStats = admin.topics().getPartitionedStats(topic, false, true, true); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); - assertEquals(topicStats.subscriptions.get(subName).backlogSize, 470); - assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 5); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get(subName).getBacklogSize(), 470); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklogNoDelayed(), 5); for (int i = 0; i < 5; i++) { consumer.acknowledge(consumer.receive()); @@ -1400,9 +1411,9 @@ public void testBacklogNoDelayedForPartitionedTopic() throws PulsarClientExcepti // Wait the ack send. Thread.sleep(500); topicStats = admin.topics().getPartitionedStats(topic, false, true, true); - assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 5); - assertEquals(topicStats.subscriptions.get(subName).backlogSize, 238); - assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 0); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 5); + assertEquals(topicStats.getSubscriptions().get(subName).getBacklogSize(), 238); + assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklogNoDelayed(), 0); } @Test @@ -1443,8 +1454,8 @@ public void testListOfNamespaceBundles() throws Exception { admin.namespaces().createNamespace("prop-xyz2/ns1", 10); admin.namespaces().setNamespaceReplicationClusters("prop-xyz2/ns1", Sets.newHashSet("test")); admin.namespaces().createNamespace("prop-xyz2/test/ns2", 10); - assertEquals(admin.namespaces().getBundles("prop-xyz2/ns1").numBundles, 10); - assertEquals(admin.namespaces().getBundles("prop-xyz2/test/ns2").numBundles, 10); + assertEquals(admin.namespaces().getBundles("prop-xyz2/ns1").getNumBundles(), 10); + assertEquals(admin.namespaces().getBundles("prop-xyz2/test/ns2").getNumBundles(), 10); } @Test @@ -1467,14 +1478,17 @@ public void testForceDeleteNamespace() throws Exception { @Test public void testUpdateClusterWithProxyUrl() throws Exception { - ClusterDataImpl cluster = new ClusterDataImpl(pulsar.getWebServiceAddress()); + ClusterData cluster = ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); String clusterName = "test2"; admin.clusters().createCluster(clusterName, cluster); Assert.assertEquals(admin.clusters().getCluster(clusterName), cluster); // update - cluster.setProxyServiceUrl("proxy"); - cluster.setProxyProtocol(ProxyProtocol.SNI); + cluster = ClusterData.builder() + .serviceUrl(pulsar.getWebServiceAddress()) + .proxyServiceUrl("proxy") + .proxyProtocol(ProxyProtocol.SNI) + .build(); admin.clusters().updateCluster(clusterName, cluster); Assert.assertEquals(admin.clusters().getCluster(clusterName), cluster); } @@ -1484,7 +1498,7 @@ public void testMaxNamespacesPerTenant() throws Exception { super.internalCleanup(); conf.setMaxNamespacesPerTenant(2); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1500,7 +1514,7 @@ public void testMaxNamespacesPerTenant() throws Exception { super.internalCleanup(); conf.setMaxNamespacesPerTenant(0); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); for (int i = 0; i < 10; i++) { admin.namespaces().createNamespace("testTenant/ns-" + i, Sets.newHashSet("test")); @@ -1512,7 +1526,7 @@ public void testMaxTopicsPerNamespace() throws Exception { super.internalCleanup(); conf.setMaxTopicsPerNamespace(10); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1535,7 +1549,7 @@ public void testMaxTopicsPerNamespace() throws Exception { super.internalCleanup(); conf.setMaxTopicsPerNamespace(0); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); for (int i = 0; i < 10; ++i) { @@ -1549,7 +1563,7 @@ public void testMaxTopicsPerNamespace() throws Exception { conf.setDefaultNumPartitions(3); conf.setAllowAutoTopicCreationType("partitioned"); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1568,7 +1582,7 @@ public void testMaxTopicsPerNamespace() throws Exception { conf.setMaxTopicsPerNamespace(3); conf.setAllowAutoTopicCreationType("non-partitioned"); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1603,7 +1617,7 @@ public void testMaxSubscriptionsPerTopic() throws Exception { conf.setMaxSubscriptionsPerTopic(2); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1628,7 +1642,7 @@ public void testMaxSubscriptionsPerTopic() throws Exception { conf.setMaxSubscriptionsPerTopic(0); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1644,7 +1658,7 @@ public void testMaxSubscriptionsPerTopic() throws Exception { conf.setMaxSubscriptionsPerTopic(2); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace("testTenant/ns1", Sets.newHashSet("test")); @@ -1753,7 +1767,7 @@ public void testMaxSubPerTopicPriority() throws Exception { conf.setMaxSubscriptionsPerTopic(brokerLevelMaxSub); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); final String myNamespace = "prop-xyz/ns" + UUID.randomUUID(); @@ -1810,7 +1824,7 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { conf.setMaxProducersPerTopic(maxProducersPerTopic); super.internalSetup(); //init namespace - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); final String myNamespace = "prop-xyz/ns" + UUID.randomUUID(); @@ -1863,7 +1877,7 @@ public void testMaxConsumersPerTopicUnlimited() throws Exception { conf.setMaxConsumersPerTopic(maxConsumersPerTopic); super.internalSetup(); //init namespace - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("prop-xyz", tenantInfo); final String myNamespace = "prop-xyz/ns" + UUID.randomUUID(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTlsAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTlsAuthTest.java index 6fe4033f2c154..fb39aae77e281 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTlsAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTlsAuthTest.java @@ -48,6 +48,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.tls.NoopHostnameVerifier; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterDataImpl; @@ -96,7 +97,7 @@ public void setup() throws Exception { super.internalSetup(); PulsarAdmin admin = buildAdminClient("admin"); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java index ee163843b5bbd..db6cacab8eb7c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java @@ -43,6 +43,7 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.Map; @@ -77,15 +78,19 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.BrokerInfo; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.ResourceQuota; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.stats.AllocatorStats; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -218,7 +223,7 @@ public void clusters() throws Exception { assertEquals(clusters.getClusters(), Lists.newArrayList()); verify(clusters, never()).validateSuperUserAccess(); - clusters.createCluster("use", new ClusterDataImpl("http://broker.messaging.use.example.com")); + clusters.createCluster("use", ClusterDataImpl.builder().serviceUrl("http://broker.messaging.use.example.com:8080").build()); verify(clusters, times(1)).validateSuperUserAccess(); // ensure to read from ZooKeeper directly //clusters.clustersListCache().clear(); @@ -226,7 +231,7 @@ public void clusters() throws Exception { // Check creating existing cluster try { - clusters.createCluster("use", new ClusterDataImpl("http://broker.messaging.use.example.com")); + clusters.createCluster("use", ClusterDataImpl.builder().serviceUrl("http://broker.messaging.use.example.com:8080").build()); fail("should have failed"); } catch (RestException e) { assertEquals(e.getResponse().getStatus(), Status.CONFLICT.getStatusCode()); @@ -240,13 +245,13 @@ public void clusters() throws Exception { assertEquals(e.getResponse().getStatus(), Status.NOT_FOUND.getStatusCode()); } - assertEquals(clusters.getCluster("use"), new ClusterDataImpl("http://broker.messaging.use.example.com")); + assertEquals(clusters.getCluster("use"), ClusterDataImpl.builder().serviceUrl("http://broker.messaging.use.example.com:8080").build()); verify(clusters, times(4)).validateSuperUserAccess(); - clusters.updateCluster("use", new ClusterDataImpl("http://new-broker.messaging.use.example.com")); + clusters.updateCluster("use", ClusterDataImpl.builder().serviceUrl("http://new-broker.messaging.use.example.com:8080").build()); verify(clusters, times(5)).validateSuperUserAccess(); - assertEquals(clusters.getCluster("use"), new ClusterDataImpl("http://new-broker.messaging.use.example.com")); + assertEquals(clusters.getCluster("use"), ClusterData.builder().serviceUrl("http://new-broker.messaging.use.example.com:8080").build()); verify(clusters, times(6)).validateSuperUserAccess(); try { @@ -255,17 +260,19 @@ public void clusters() throws Exception { } catch (RestException e) { assertEquals(e.getResponse().getStatus(), 404); } - NamespaceIsolationDataImpl policyData = new NamespaceIsolationDataImpl(); - policyData.namespaces = new ArrayList(); - policyData.namespaces.add("dummy/colo/ns"); - policyData.primary = new ArrayList(); - policyData.primary.add("localhost" + ":" + pulsar.getListenPortHTTP()); - policyData.secondary = new ArrayList(); - policyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - policyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - policyData.auto_failover_policy.parameters = new HashMap(); - policyData.auto_failover_policy.parameters.put("min_limit", "1"); - policyData.auto_failover_policy.parameters.put("usage_threshold", "90"); + + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "90"); + + NamespaceIsolationDataImpl policyData = NamespaceIsolationDataImpl.builder() + .namespaces(Collections.singletonList("dummy/colo/ns")) + .primary(Collections.singletonList("localhost" + ":" + pulsar.getListenPortHTTP())) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); AsyncResponse response = mock(AsyncResponse.class); clusters.setNamespaceIsolationPolicy(response,"use", "policy1", policyData); clusters.getNamespaceIsolationPolicies("use"); @@ -292,7 +299,7 @@ public void clusters() throws Exception { } try { - clusters.updateCluster("use", new ClusterDataImpl()); + clusters.updateCluster("use", ClusterDataImpl.builder().build()); fail("should have failed"); } catch (RestException e) { assertEquals(e.getResponse().getStatus(), 404); @@ -312,7 +319,7 @@ public void clusters() throws Exception { }); configurationCache.clustersListCache().clear(); // clear caches to load data from metadata-store again - MetadataCacheImpl clusterCache = (MetadataCacheImpl) pulsar.getPulsarResources() + MetadataCacheImpl clusterCache = (MetadataCacheImpl) pulsar.getPulsarResources() .getClusterResources().getCache(); MetadataCacheImpl isolationPolicyCache = (MetadataCacheImpl) pulsar.getPulsarResources() .getNamespaceResources().getIsolationPolicies().getCache(); @@ -331,7 +338,7 @@ public void clusters() throws Exception { && path.equals("/admin/clusters/test"); }); try { - clusters.createCluster("test", new ClusterDataImpl("http://broker.messaging.test.example.com")); + clusters.createCluster("test", ClusterDataImpl.builder().serviceUrl("http://broker.messaging.test.example.com:8080").build()); fail("should have failed"); } catch (RestException e) { assertEquals(e.getResponse().getStatus(), Status.INTERNAL_SERVER_ERROR.getStatusCode()); @@ -344,7 +351,7 @@ public void clusters() throws Exception { clusterCache.invalidateAll(); store.invalidateAll(); try { - clusters.updateCluster("test", new ClusterDataImpl("http://broker.messaging.test.example.com")); + clusters.updateCluster("test", ClusterDataImpl.builder().serviceUrl("http://broker.messaging.test.example.com").build()); fail("should have failed"); } catch (RestException e) { assertEquals(e.getResponse().getStatus(), Status.INTERNAL_SERVER_ERROR.getStatusCode()); @@ -390,7 +397,7 @@ public void clusters() throws Exception { // Check name validations try { - clusters.createCluster("bf@", new ClusterDataImpl("http://dummy.messaging.example.com")); + clusters.createCluster("bf@", ClusterDataImpl.builder().serviceUrl("http://dummy.messaging.example.com").build()); fail("should have filed"); } catch (RestException e) { assertEquals(e.getResponse().getStatus(), Status.PRECONDITION_FAILED.getStatusCode()); @@ -398,10 +405,16 @@ public void clusters() throws Exception { // Check authentication and listener name try { - clusters.createCluster("auth", new ClusterDataImpl("http://dummy.web.example.com", "", - "http://dummy.messaging.example.com", "", - "authenticationPlugin", "authenticationParameters", "listenerName")); - ClusterDataImpl cluster = clusters.getCluster("auth"); + clusters.createCluster("auth", ClusterDataImpl.builder() + .serviceUrl("http://dummy.web.example.com") + .serviceUrlTls("") + .brokerServiceUrl("http://dummy.messaging.example.com") + .brokerServiceUrlTls("") + .authenticationPlugin("authenticationPlugin") + .authenticationParameters("authenticationParameters") + .listenerName("listenerName") + .build()); + ClusterData cluster = clusters.getCluster("auth"); assertEquals("authenticationPlugin", cluster.getAuthenticationPlugin()); assertEquals("authenticationParameters", cluster.getAuthenticationParameters()); assertEquals("listenerName", cluster.getListenerName()); @@ -426,11 +439,14 @@ public void properties() throws Throwable { verify(properties, times(1)).validateSuperUserAccess(); // create local cluster - clusters.createCluster(configClusterName, new ClusterDataImpl()); + clusters.createCluster(configClusterName, ClusterDataImpl.builder().build()); Set allowedClusters = Sets.newHashSet(); allowedClusters.add(configClusterName); - TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), allowedClusters); + TenantInfoImpl tenantInfo = TenantInfoImpl.builder() + .adminRoles(Sets.newHashSet("role1", "role2")) + .allowedClusters(allowedClusters) + .build(); response = asynRequests(ctx -> properties.createTenant(ctx, "test-property", tenantInfo)); verify(properties, times(2)).validateSuperUserAccess(); @@ -442,7 +458,10 @@ public void properties() throws Throwable { assertEquals(response, tenantInfo); verify(properties, times(4)).validateSuperUserAccess(); - final TenantInfoImpl newPropertyAdmin = new TenantInfoImpl(Sets.newHashSet("role1", "other-role"), allowedClusters); + final TenantInfoImpl newPropertyAdmin = TenantInfoImpl.builder() + .adminRoles(Sets.newHashSet("role1", "other-role")) + .allowedClusters(allowedClusters) + .build(); response = asynRequests(ctx -> properties.updateTenant(ctx, "test-property", newPropertyAdmin)); verify(properties, times(5)).validateSuperUserAccess(); @@ -487,7 +506,7 @@ public void properties() throws Throwable { } // clear caches to load data from metadata-store again - MetadataCacheImpl cache = (MetadataCacheImpl) pulsar.getPulsarResources() + MetadataCacheImpl cache = (MetadataCacheImpl) pulsar.getPulsarResources() .getTenantResources().getCache(); AbstractMetadataStore store = (AbstractMetadataStore) cache.getStore(); cache.invalidateAll(); @@ -564,10 +583,13 @@ public void properties() throws Throwable { assertEquals(response, Lists.newArrayList()); // Create a namespace to test deleting a non-empty property - TenantInfoImpl newPropertyAdmin2 = new TenantInfoImpl(Sets.newHashSet("role1", "other-role"), Sets.newHashSet("use")); + TenantInfoImpl newPropertyAdmin2 = TenantInfoImpl.builder() + .adminRoles(Sets.newHashSet("role1", "other-role")) + .allowedClusters(Sets.newHashSet("use")) + .build(); response = asynRequests(ctx -> properties.createTenant(ctx, "my-tenant", newPropertyAdmin2)); - namespaces.createNamespace("my-tenant", "use", "my-namespace", new BundlesData()); + namespaces.createNamespace("my-tenant", "use", "my-namespace", BundlesData.builder().build()); try { response = asynRequests(ctx -> properties.deleteTenant(ctx, "my-tenant", false)); @@ -595,7 +617,10 @@ public void properties() throws Throwable { // Check tenantInfo with empty cluster String blankCluster = ""; Set blankClusters = Sets.newHashSet(blankCluster); - TenantInfoImpl tenantWithEmptyCluster = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), blankClusters); + TenantInfoImpl tenantWithEmptyCluster = TenantInfoImpl.builder() + .adminRoles(Sets.newHashSet("role1", "role2")) + .allowedClusters(blankClusters) + .build(); try { response = asynRequests(ctx -> properties.createTenant(ctx, "tenant-config-is-empty", tenantWithEmptyCluster)); fail("should have failed"); @@ -606,7 +631,10 @@ public void properties() throws Throwable { // Check tenantInfo contains empty cluster Set containBlankClusters = Sets.newHashSet(blankCluster); containBlankClusters.add(configClusterName); - TenantInfoImpl tenantContainEmptyCluster = new TenantInfoImpl(Sets.newHashSet(), containBlankClusters); + TenantInfoImpl tenantContainEmptyCluster = TenantInfoImpl.builder() + .adminRoles(Sets.newHashSet()) + .allowedClusters(containBlankClusters) + .build(); try { response = asynRequests(ctx -> properties.createTenant(ctx, "tenant-config-contain-empty", tenantContainEmptyCluster)); fail("should have failed"); @@ -624,8 +652,10 @@ public void properties() throws Throwable { @Test public void brokers() throws Exception { - clusters.createCluster("use", new ClusterDataImpl("http://broker.messaging.use.example.com", - "https://broker.messaging.use.example.com:4443")); + clusters.createCluster("use", ClusterDataImpl.builder() + .serviceUrl("http://broker.messaging.use.example.com") + .serviceUrlTls("https://broker.messaging.use.example.com:4443") + .build()); URI requestUri = new URI( "http://broker.messaging.use.example.com:8080/admin/brokers/use"); @@ -681,9 +711,10 @@ public void resourceQuotas() throws Exception { } // create policies - TenantInfoImpl admin = new TenantInfoImpl(); - admin.getAllowedClusters().add(cluster); - ClusterDataImpl clusterData = new ClusterDataImpl(cluster); + TenantInfoImpl admin = TenantInfoImpl.builder() + .allowedClusters(Collections.singleton(cluster)) + .build(); + ClusterDataImpl clusterData = ClusterDataImpl.builder().serviceUrl(cluster).build(); clusters.createCluster(cluster, clusterData ); asynRequests(ctx -> properties.createTenant(ctx, property, admin)); @@ -740,8 +771,9 @@ public void persistentTopics() throws Exception { doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace)); doReturn("client-id").when(resourceQuotas).clientAppId(); // create policies - TenantInfoImpl admin = new TenantInfoImpl(); - admin.getAllowedClusters().add(cluster); + TenantInfo admin = TenantInfo.builder() + .allowedClusters(Collections.singleton(cluster)) + .build(); ZkUtils.createFullPathOptimistic(mockZooKeeperGlobal, PulsarWebResource.path(POLICIES, property, cluster, namespace), ObjectMapperFactory.getThreadLocal().writeValueAsBytes(new Policies()), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BookiesApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BookiesApiTest.java index 2d568c2f7dbff..644d47df7678d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BookiesApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BookiesApiTest.java @@ -68,14 +68,14 @@ public void testBasic() throws Exception { } // update the bookie info - BookieInfo newInfo0 = new BookieInfo( - "/rack1", - "127.0.0.1" - ); - BookieInfo newInfo1 = new BookieInfo( - "/rack1", - "127.0.0.2" - ); + BookieInfo newInfo0 = BookieInfo.builder() + .rack("/rack1") + .hostname("127.0.0.1") + .build(); + BookieInfo newInfo1 = BookieInfo.builder() + .rack("/rack1") + .hostname("127.0.0.2") + .build(); admin.bookies().updateBookieRackInfo(bookie0, "default", newInfo0); BookieInfo readInfo0 = admin.bookies().getBookieRackInfo(bookie0); assertEquals(newInfo0, readInfo0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerAdminClientTlsAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerAdminClientTlsAuthTest.java index 847c0800837d5..9edb0e78aeb19 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerAdminClientTlsAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerAdminClientTlsAuthTest.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -129,14 +130,14 @@ public void testPersistentList() throws Exception { /***** Broker 2 Started *****/ try (PulsarAdmin admin = buildAdminClient("superproxy")) { - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("tenant", new TenantInfoImpl(ImmutableSet.of("admin"), ImmutableSet.of("test"))); } try (PulsarAdmin admin = buildAdminClient("admin")) { Policies policies = new Policies(); - policies.bundles = new BundlesData(4); + policies.bundles = BundlesData.builder().numBundles(4).build(); policies.replication_clusters = ImmutableSet.of("test"); admin.namespaces().createNamespace("tenant/ns", policies); try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java index fbdcbf487c6ca..e0d1720dc1f9a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java @@ -69,16 +69,16 @@ public void createSubscriptionSingleTopic() throws Exception { p1.send("test-2".getBytes()); MessageId m3 = p1.send("test-3".getBytes()); - assertEquals(admin.topics().getStats(topic).subscriptions.get("sub-1").msgBacklog, 3); + assertEquals(admin.topics().getStats(topic).getSubscriptions().get("sub-1").getMsgBacklog(), 3); admin.topics().createSubscription(topic, "sub-2", MessageId.latest); - assertEquals(admin.topics().getStats(topic).subscriptions.get("sub-2").msgBacklog, 0); + assertEquals(admin.topics().getStats(topic).getSubscriptions().get("sub-2").getMsgBacklog(), 0); admin.topics().createSubscription(topic, "sub-3", MessageId.earliest); - assertEquals(admin.topics().getStats(topic).subscriptions.get("sub-3").msgBacklog, 3); + assertEquals(admin.topics().getStats(topic).getSubscriptions().get("sub-3").getMsgBacklog(), 3); admin.topics().createSubscription(topic, "sub-5", m3); - assertEquals(admin.topics().getStats(topic).subscriptions.get("sub-5").msgBacklog, 1); + assertEquals(admin.topics().getStats(topic).getSubscriptions().get("sub-5").getMsgBacklog(), 1); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java index 17435936230f9..3a9a89b644011 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java @@ -33,6 +33,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; @@ -57,7 +58,7 @@ public void setup() throws Exception { mockPulsarSetup.setup(); // Setup namespaces - admin.clusters().createCluster("use", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/use/ns1"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index 132d53e2c6bd8..1f840b1177e1e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -84,6 +84,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PersistencePolicies; @@ -92,6 +93,7 @@ import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SubscribeRate; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl; @@ -166,17 +168,17 @@ public void setup() throws Exception { doReturn(null).when(namespaces).clientAuthData(); doReturn(Sets.newTreeSet(Lists.newArrayList("use", "usw", "usc", "global"))).when(namespaces).clusters(); - admin.clusters().createCluster("use", new ClusterDataImpl("http://broker-use.com:8080")); - admin.clusters().createCluster("usw", new ClusterDataImpl("http://broker-usw.com:8080")); - admin.clusters().createCluster("usc", new ClusterDataImpl("http://broker-usc.com:8080")); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl("http://broker-use.com:8080").build()); + admin.clusters().createCluster("usw", ClusterData.builder().serviceUrl("http://broker-usw.com:8080").build()); + admin.clusters().createCluster("usc", ClusterData.builder().serviceUrl("http://broker-usc.com:8080").build()); admin.tenants().createTenant(this.testTenant, new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use", "usc", "usw"))); admin.tenants().createTenant(this.testOtherTenant, new TenantInfoImpl(Sets.newHashSet("role3", "role4"), Sets.newHashSet("use", "usc", "usw"))); - createTestNamespaces(this.testLocalNamespaces, new BundlesData()); + createTestNamespaces(this.testLocalNamespaces, BundlesData.builder().build()); createGlobalTestNamespaces(this.testTenant, this.testGlobalNamespaces.get(0).getLocalName(), - new BundlesData()); + BundlesData.builder().build()); doThrow(new RestException(Status.UNAUTHORIZED, "unauthorized")).when(namespaces) .validateTenantOperation(this.testOtherTenant, null); @@ -202,7 +204,8 @@ public void cleanup() throws Exception { @Test public void testCreateNamespaces() throws Exception { try { - namespaces.createNamespace(this.testTenant, "other-colo", "my-namespace", new BundlesData()); + namespaces.createNamespace(this.testTenant, "other-colo", "my-namespace", + BundlesData.builder().build()); fail("should have failed"); } catch (RestException e) { // Ok, cluster doesn't exist @@ -212,24 +215,27 @@ public void testCreateNamespaces() throws Exception { nsnames.add(NamespaceName.get(this.testTenant, "use", "create-namespace-1")); nsnames.add(NamespaceName.get(this.testTenant, "use", "create-namespace-2")); nsnames.add(NamespaceName.get(this.testTenant, "usc", "create-other-namespace-1")); - createTestNamespaces(nsnames, new BundlesData()); + createTestNamespaces(nsnames, BundlesData.builder().build()); try { - namespaces.createNamespace(this.testTenant, "use", "create-namespace-1", new BundlesData()); + namespaces.createNamespace(this.testTenant, "use", "create-namespace-1", + BundlesData.builder().build()); fail("should have failed"); } catch (RestException e) { // Ok, namespace already exists } try { - namespaces.createNamespace("non-existing-tenant", "use", "create-namespace-1", new BundlesData()); + namespaces.createNamespace("non-existing-tenant", "use", "create-namespace-1", + BundlesData.builder().build()); fail("should have failed"); } catch (RestException e) { // Ok, tenant doesn't exist } try { - namespaces.createNamespace(this.testTenant, "use", "create-namespace-#", new BundlesData()); + namespaces.createNamespace(this.testTenant, "use", "create-namespace-#", + BundlesData.builder().build()); fail("should have failed"); } catch (RestException e) { // Ok, invalid namespace name @@ -241,7 +247,7 @@ public void testCreateNamespaces() throws Exception { && path.equals("/admin/policies/my-tenant/use/my-namespace-3"); }); try { - namespaces.createNamespace(this.testTenant, "use", "my-namespace-3", new BundlesData()); + namespaces.createNamespace(this.testTenant, "use", "my-namespace-3", BundlesData.builder().build()); fail("should have failed"); } catch (RestException e) { // Ok @@ -288,7 +294,7 @@ public void testGetNamespaces() throws Exception { && path.equals("/admin/policies/my-tenant"); }); // clear caches to load data from metadata-store again - MetadataCacheImpl tenantCache = (MetadataCacheImpl) pulsar.getPulsarResources() + MetadataCacheImpl tenantCache = (MetadataCacheImpl) pulsar.getPulsarResources() .getTenantResources().getCache(); AbstractMetadataStore store = (AbstractMetadataStore) tenantCache.getStore(); tenantCache.invalidateAll(); @@ -319,32 +325,32 @@ public void testGrantAndRevokePermissions() throws Exception { assertEquals(namespaces.getPolicies(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies); assertEquals(namespaces.getPermissions(this.testTenant, this.testLocalCluster, - this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.namespace_auth); + this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.getNamespaceAuthentication()); namespaces.grantPermissionOnNamespace(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName(), "my-role", EnumSet.of(AuthAction.produce)); - expectedPolicies.auth_policies.namespace_auth.put("my-role", EnumSet.of(AuthAction.produce)); + expectedPolicies.auth_policies.getNamespaceAuthentication().put("my-role", EnumSet.of(AuthAction.produce)); assertEquals(namespaces.getPolicies(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies); assertEquals(namespaces.getPermissions(this.testTenant, this.testLocalCluster, - this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.namespace_auth); + this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.getNamespaceAuthentication()); namespaces.grantPermissionOnNamespace(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName(), "other-role", EnumSet.of(AuthAction.consume)); - expectedPolicies.auth_policies.namespace_auth.put("other-role", EnumSet.of(AuthAction.consume)); + expectedPolicies.auth_policies.getNamespaceAuthentication().put("other-role", EnumSet.of(AuthAction.consume)); assertEquals(namespaces.getPolicies(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies); assertEquals(namespaces.getPermissions(this.testTenant, this.testLocalCluster, - this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.namespace_auth); + this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.getNamespaceAuthentication()); namespaces.revokePermissionsOnNamespace(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName(), "my-role"); - expectedPolicies.auth_policies.namespace_auth.remove("my-role"); + expectedPolicies.auth_policies.getNamespaceAuthentication().remove("my-role"); assertEquals(namespaces.getPolicies(this.testTenant, this.testLocalCluster, this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies); assertEquals(namespaces.getPermissions(this.testTenant, this.testLocalCluster, - this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.namespace_auth); + this.testLocalNamespaces.get(0).getLocalName()), expectedPolicies.auth_policies.getNamespaceAuthentication()); // Non-existing namespaces try { @@ -612,7 +618,11 @@ public void testGlobalNamespaceReplicationConfiguration() throws Exception { @Test public void testGetBundles() throws Exception { - BundlesData bundle = new BundlesData(Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff"); + BundlesData bundle = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, "test-bundled-namespace-1", bundle); BundlesData responseData = namespaces.getBundlesData(testTenant, this.testLocalCluster, "test-bundled-namespace-1"); @@ -771,7 +781,11 @@ public void testDeleteNamespaces() throws Exception { public void testDeleteNamespaceWithBundles() throws Exception { URL localWebServiceUrl = new URL(pulsar.getSafeWebServiceAddress()); String bundledNsLocal = "test-bundled-namespace-1"; - BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff"); + BundlesData bundleData = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, bundledNsLocal, bundleData); final NamespaceName testNs = NamespaceName.get(this.testTenant, this.testLocalCluster, bundledNsLocal); @@ -875,7 +889,11 @@ public void testUnloadNamespaces() throws Exception { public void testSplitBundles() throws Exception { URL localWebServiceUrl = new URL(pulsar.getSafeWebServiceAddress()); String bundledNsLocal = "test-bundled-namespace-1"; - BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0xffffffff"); + BundlesData bundleData = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, bundledNsLocal, bundleData); final NamespaceName testNs = NamespaceName.get(this.testTenant, this.testLocalCluster, bundledNsLocal); @@ -896,10 +914,10 @@ public void testSplitBundles() throws Exception { // verify split bundles BundlesData bundlesData = namespaces.getBundlesData(testTenant, testLocalCluster, bundledNsLocal); assertNotNull(bundlesData); - assertEquals(bundlesData.boundaries.size(), 3); - assertEquals(bundlesData.boundaries.get(0), "0x00000000"); - assertEquals(bundlesData.boundaries.get(1), "0x7fffffff"); - assertEquals(bundlesData.boundaries.get(2), "0xffffffff"); + assertEquals(bundlesData.getBoundaries().size(), 3); + assertEquals(bundlesData.getBoundaries().get(0), "0x00000000"); + assertEquals(bundlesData.getBoundaries().get(1), "0x7fffffff"); + assertEquals(bundlesData.getBoundaries().get(2), "0xffffffff"); } catch (RestException re) { assertEquals(re.getResponse().getStatus(), Status.PRECONDITION_FAILED.getStatusCode()); } @@ -909,8 +927,11 @@ public void testSplitBundles() throws Exception { public void testSplitBundleWithUnDividedRange() throws Exception { URL localWebServiceUrl = new URL(pulsar.getSafeWebServiceAddress()); String bundledNsLocal = "test-bundled-namespace-1"; - BundlesData bundleData = new BundlesData( - Lists.newArrayList("0x00000000", "0x08375b1a", "0x08375b1b", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0x08375b1a", "0x08375b1b", "0xffffffff"); + BundlesData bundleData = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, bundledNsLocal, bundleData); final NamespaceName testNs = NamespaceName.get(this.testTenant, this.testLocalCluster, bundledNsLocal); @@ -933,7 +954,11 @@ public void testSplitBundleWithUnDividedRange() throws Exception { public void testUnloadNamespaceWithBundles() throws Exception { URL localWebServiceUrl = new URL(pulsar.getSafeWebServiceAddress()); String bundledNsLocal = "test-bundled-namespace-1"; - BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff"); + BundlesData bundleData = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, bundledNsLocal, bundleData); final NamespaceName testNs = NamespaceName.get(this.testTenant, this.testLocalCluster, bundledNsLocal); @@ -1002,7 +1027,11 @@ public void testRetention() throws Exception { try { URL localWebServiceUrl = new URL(pulsar.getSafeWebServiceAddress()); String bundledNsLocal = "test-bundled-namespace-1"; - BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0xffffffff"); + BundlesData bundleData = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, bundledNsLocal, bundleData); final NamespaceName testNs = NamespaceName.get(this.testTenant, this.testLocalCluster, bundledNsLocal); mockWebUrl(localWebServiceUrl, testNs); @@ -1060,7 +1089,11 @@ public void testPersistenceUnauthorized() throws Exception { public void testValidateTopicOwnership() throws Exception { URL localWebServiceUrl = new URL(pulsar.getSafeWebServiceAddress()); String bundledNsLocal = "test-bundled-namespace-1"; - BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff")); + List boundaries = Lists.newArrayList("0x00000000", "0xffffffff"); + BundlesData bundleData = BundlesData.builder() + .boundaries(boundaries) + .numBundles(boundaries.size() - 1) + .build(); createBundledTestNamespaces(this.testTenant, this.testLocalCluster, bundledNsLocal, bundleData); final NamespaceName testNs = NamespaceName.get(this.testTenant, this.testLocalCluster, bundledNsLocal); OwnershipCache MockOwnershipCache = spy(pulsar.getNamespaceService().getOwnershipCache()); @@ -1099,7 +1132,7 @@ public void testDeleteNamespace() throws Exception { final String namespace = this.testTenant + "/use/deleteNs"; admin.namespaces().createNamespace(namespace, 100); - assertEquals(admin.namespaces().getPolicies(namespace).bundles.numBundles, 100); + assertEquals(admin.namespaces().getPolicies(namespace).bundles.getNumBundles(), 100); // (1) Force topic creation and namespace being loaded final String topicName = "persistent://" + namespace + "/my-topic"; @@ -1477,7 +1510,7 @@ public void testMaxTopicsPerNamespace() throws Exception { super.internalSetup(); String namespace = "testTenant/ns1"; - admin.clusters().createCluster("use", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use")); admin.tenants().createTenant("testTenant", tenantInfo); @@ -1527,7 +1560,7 @@ public void testMaxTopicsPerNamespace() throws Exception { conf.setAllowAutoTopicCreationType("partitioned"); super.internalSetup(); - admin.clusters().createCluster("use", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace(namespace, Sets.newHashSet("use")); admin.namespaces().setMaxTopicsPerNamespace(namespace, 10); @@ -1557,7 +1590,7 @@ public void testMaxTopicsPerNamespace() throws Exception { conf.setAllowAutoTopicCreationType("non-partitioned"); super.internalSetup(); - admin.clusters().createCluster("use", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("testTenant", tenantInfo); admin.namespaces().createNamespace(namespace, Sets.newHashSet("use")); admin.namespaces().setMaxTopicsPerNamespace(namespace, 3); 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 7985beff1061b..39a99d2784d69 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 @@ -65,6 +65,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -128,8 +129,8 @@ protected void setup() throws Exception { doReturn(mock(AuthenticationDataHttps.class)).when(nonPersistentTopic).clientAuthData(); - admin.clusters().createCluster("use", new ClusterDataImpl("http://broker-use.com:8080")); - admin.clusters().createCluster("test", new ClusterDataImpl("http://broker-use.com:8080")); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl("http://broker-use.com:8080").build()); + admin.clusters().createCluster("test",ClusterData.builder().serviceUrl("http://broker-use.com:8080").build()); admin.tenants().createTenant(this.testTenant, new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet(testLocalCluster, "test"))); admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Sets.newHashSet(testLocalCluster, "test")); @@ -237,7 +238,7 @@ public void testCreateSubscriptions() throws Exception{ verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); TopicStats topicStats = persistentTopics.getStats(testTenant, testNamespace, testLocalTopicName, true, true, false); - long msgBacklog = topicStats.subscriptions.get(SUB_EARLIEST).msgBacklog; + long msgBacklog = topicStats.getSubscriptions().get(SUB_EARLIEST).getMsgBacklog(); System.out.println("Message back log for " + SUB_EARLIEST + " is :" + msgBacklog); Assert.assertEquals(msgBacklog, numberOfMessages); @@ -250,7 +251,7 @@ public void testCreateSubscriptions() throws Exception{ verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); topicStats = persistentTopics.getStats(testTenant, testNamespace, testLocalTopicName, true, true, false); - msgBacklog = topicStats.subscriptions.get(SUB_LATEST).msgBacklog; + msgBacklog = topicStats.getSubscriptions().get(SUB_LATEST).getMsgBacklog(); System.out.println("Message back log for " + SUB_LATEST + " is :" + msgBacklog); Assert.assertEquals(msgBacklog, 0); @@ -263,7 +264,7 @@ public void testCreateSubscriptions() throws Exception{ verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); topicStats = persistentTopics.getStats(testTenant, testNamespace, testLocalTopicName, true, true, false); - msgBacklog = topicStats.subscriptions.get(SUB_NONE_MESSAGE_ID).msgBacklog; + msgBacklog = topicStats.getSubscriptions().get(SUB_NONE_MESSAGE_ID).getMsgBacklog(); System.out.println("Message back log for " + SUB_NONE_MESSAGE_ID + " is :" + msgBacklog); Assert.assertEquals(msgBacklog, 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java index 9183d95b1a937..40d82ca450e5d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.broker.admin.v2.ResourceGroups; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ResourceGroup; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -173,7 +174,7 @@ public void testNamespaceResourceGroup() throws Exception { } private void prepareData() throws PulsarAdminException { - admin.clusters().createCluster(testCluster, new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster(testCluster, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant(testTenant, new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet(testCluster))); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java index 710920a9966f0..afe1144712d2c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; @@ -57,7 +58,7 @@ protected void setup() throws Exception { this.conf.setTtlDurationDefaultInSeconds(3600); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant(this.testTenant, tenantInfo); admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Sets.newHashSet("test")); @@ -130,7 +131,7 @@ public void testTopicPolicyDisabled() throws Exception { this.conf.setTopicLevelPoliciesEnabled(false); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant(this.testTenant, tenantInfo); admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesDisableTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesDisableTest.java index 2420a5b2ad873..5e9967bfe7988 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesDisableTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesDisableTest.java @@ -23,6 +23,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.PersistencePolicies; @@ -55,7 +56,7 @@ protected void setup() throws Exception { this.conf.setTopicLevelPoliciesEnabled(false); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant(this.testTenant, tenantInfo); admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Sets.newHashSet("test")); @@ -70,7 +71,10 @@ public void cleanup() throws Exception { @Test public void testBacklogQuotaDisabled() { - BacklogQuota backlogQuota = new BacklogQuota(1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); log.info("Backlog quota: {} will set to the topic: {}", backlogQuota, testTopic); try { @@ -137,7 +141,7 @@ public void testPersistenceDisabled() { @Test public void testDispatchRateDisabled() throws Exception { - DispatchRate dispatchRate = new DispatchRate(); + DispatchRate dispatchRate = DispatchRate.builder().build(); log.info("Dispatch Rate: {} will set to the topic: {}", dispatchRate, testTopic); try { @@ -157,8 +161,11 @@ public void testDispatchRateDisabled() throws Exception { @Test public void testSubscriptionDispatchRateDisabled() throws Exception { - DispatchRate dispatchRate = new DispatchRate(1000, - 1020*1024, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(1000) + .dispatchThrottlingRateInMsg(1020*1024) + .ratePeriodInSecond(1) + .build(); log.info("Dispatch Rate: {} will set to the topic: {}", dispatchRate, testTopic); try { 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 c227870619504..276153d4cdf61 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 @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; @@ -98,7 +99,7 @@ protected void setup() throws Exception { this.conf.setDefaultNumberOfNamespaceBundles(1); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant(this.testTenant, tenantInfo); admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Sets.newHashSet("test")); @@ -118,7 +119,10 @@ public void cleanup() throws Exception { @Test public void testSetBacklogQuota() throws Exception { - BacklogQuota backlogQuota = new BacklogQuota(1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); log.info("Backlog quota: {} will set to the topic: {}", backlogQuota, testTopic); Awaitility.await() @@ -141,7 +145,10 @@ public void testSetBacklogQuota() throws Exception { @Test public void testRemoveBacklogQuota() throws Exception { - BacklogQuota backlogQuota = new BacklogQuota(1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); log.info("Backlog quota: {} will set to the topic: {}", backlogQuota, testTopic); Awaitility.await() @@ -181,8 +188,10 @@ public void testCheckBacklogQuota() throws Exception { Awaitility.await() .untilAsserted(() -> Assert.assertEquals(admin.namespaces().getRetention(namespace), retentionPolicies)); - BacklogQuota backlogQuota = - new BacklogQuota(10 * 1024 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(10 * 1024 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); log.info("Backlog quota: {} will set to the topic: {}", backlogQuota, testTopic); try { admin.topics().setBacklogQuota(testTopic, backlogQuota); @@ -191,8 +200,10 @@ public void testCheckBacklogQuota() throws Exception { Assert.assertEquals(e.getStatusCode(), 412); } - backlogQuota = - new BacklogQuota(10 * 1024 * 1024 + 1, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + backlogQuota = BacklogQuota.builder() + .limitSize(10 * 1024 * 1024 + 1) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); log.info("Backlog quota: {} will set to the topic: {}", backlogQuota, testTopic); try { admin.topics().setBacklogQuota(testTopic, backlogQuota); @@ -201,8 +212,10 @@ public void testCheckBacklogQuota() throws Exception { Assert.assertEquals(e.getStatusCode(), 412); } - backlogQuota = - new BacklogQuota(10 * 1024 * 1024 - 1, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + backlogQuota = BacklogQuota.builder() + .limitSize(10 * 1024 * 1024 - 1) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); log.info("Backlog quota: {} will set to the topic: {}", backlogQuota, testTopic); admin.topics().setBacklogQuota(testTopic, backlogQuota); @@ -224,7 +237,10 @@ public void testGetBacklogQuotaApplied() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap(myNamespace), Maps.newHashMap()); Map brokerQuotaMap = ConfigHelper.backlogQuotaMap(conf); assertEquals(admin.topics().getBacklogQuotaMap(topic, true), brokerQuotaMap); - BacklogQuota namespaceQuota = new BacklogQuota(30L, BacklogQuota.RetentionPolicy.producer_exception); + BacklogQuota namespaceQuota = BacklogQuota.builder() + .limitSize(30) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(); admin.namespaces().setBacklogQuota(myNamespace, namespaceQuota); Awaitility.await().untilAsserted(() -> assertFalse(admin.namespaces().getBacklogQuotaMap(myNamespace).isEmpty())); @@ -232,7 +248,10 @@ public void testGetBacklogQuotaApplied() throws Exception { namespaceQuotaMap.put(BacklogQuota.BacklogQuotaType.destination_storage, namespaceQuota); assertEquals(admin.topics().getBacklogQuotaMap(topic, true), namespaceQuotaMap); - BacklogQuota topicQuota = new BacklogQuota(40L, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota topicQuota = BacklogQuota.builder() + .limitSize(40) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); admin.topics().setBacklogQuota(topic, topicQuota); Awaitility.await().untilAsserted(() -> assertFalse(admin.topics().getBacklogQuotaMap(topic).isEmpty())); Map topicQuotaMap = Maps.newHashMap(); @@ -255,8 +274,10 @@ public void testCheckBacklogQuotaFailed() throws Exception { Awaitility.await() .untilAsserted(() -> Assert.assertEquals(admin.namespaces().getRetention(namespace), retentionPolicies)); - BacklogQuota backlogQuota = - new BacklogQuota(10 * 1024 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(10 * 1024 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); try { admin.topics().setBacklogQuota(testTopic, backlogQuota); Assert.fail(); @@ -271,8 +292,10 @@ public void testCheckBacklogQuotaFailed() throws Exception { @Test public void testCheckRetention() throws Exception { - BacklogQuota backlogQuota = - new BacklogQuota(10 * 1024 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(10 * 1024 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build(); Awaitility.await() .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(testTopic))); @@ -387,19 +410,28 @@ public void testGetSubDispatchRateApplied() throws Exception { .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); assertNull(admin.topics().getSubscriptionDispatchRate(topic)); assertNull(admin.namespaces().getSubscriptionDispatchRate(myNamespace)); - DispatchRate brokerDispatchRate = new DispatchRate( - pulsar.getConfiguration().getDispatchThrottlingRatePerSubscriptionInMsg(), - pulsar.getConfiguration().getDispatchThrottlingRatePerSubscriptionInByte(), - 1 - ); + + DispatchRate brokerDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(conf.getDispatchThrottlingRatePerSubscriptionInMsg()) + .dispatchThrottlingRateInByte(conf.getDispatchThrottlingRatePerSubscriptionInByte()) + .ratePeriodInSecond(1) + .build(); assertEquals(admin.topics().getSubscriptionDispatchRate(topic, true), brokerDispatchRate); - DispatchRate namespaceDispatchRate = new DispatchRate(10, 11, 12); + DispatchRate namespaceDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(11) + .ratePeriodInSecond(12) + .build(); admin.namespaces().setSubscriptionDispatchRate(myNamespace, namespaceDispatchRate); Awaitility.await().untilAsserted(() -> assertNotNull(admin.namespaces().getSubscriptionDispatchRate(myNamespace))); assertEquals(admin.topics().getSubscriptionDispatchRate(topic, true), namespaceDispatchRate); - DispatchRate topicDispatchRate = new DispatchRate(20, 21, 22); + DispatchRate topicDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(20) + .dispatchThrottlingRateInByte(21) + .ratePeriodInSecond(12) + .build(); admin.topics().setSubscriptionDispatchRate(topic, topicDispatchRate); Awaitility.await().untilAsserted(() -> assertNotNull(admin.topics().getSubscriptionDispatchRate(topic))); assertEquals(admin.topics().getSubscriptionDispatchRate(topic, true), topicDispatchRate); @@ -584,19 +616,27 @@ public void testGetDispatchRateApplied() throws Exception { .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); assertNull(admin.topics().getDispatchRate(topic)); assertNull(admin.namespaces().getDispatchRate(myNamespace)); - DispatchRate brokerDispatchRate = new DispatchRate( - pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInMsg(), - pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(), - 1 - ); + DispatchRate brokerDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(conf.getDispatchThrottlingRatePerTopicInMsg()) + .dispatchThrottlingRateInByte(conf.getDispatchThrottlingRatePerTopicInByte()) + .ratePeriodInSecond(1) + .build(); assertEquals(admin.topics().getDispatchRate(topic, true), brokerDispatchRate); - DispatchRate namespaceDispatchRate = new DispatchRate(10, 11, 12); + DispatchRate namespaceDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(11) + .ratePeriodInSecond(12) + .build(); admin.namespaces().setDispatchRate(myNamespace, namespaceDispatchRate); Awaitility.await().untilAsserted(() -> assertNotNull(admin.namespaces().getDispatchRate(myNamespace))); assertEquals(admin.topics().getDispatchRate(topic, true), namespaceDispatchRate); - DispatchRate topicDispatchRate = new DispatchRate(20, 21, 22); + DispatchRate topicDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(20) + .dispatchThrottlingRateInByte(21) + .ratePeriodInSecond(22) + .build(); admin.topics().setDispatchRate(topic, topicDispatchRate); Awaitility.await().untilAsserted(() -> assertNotNull(admin.topics().getDispatchRate(topic))); assertEquals(admin.topics().getDispatchRate(topic, true), topicDispatchRate); @@ -773,7 +813,12 @@ public void testRemoveMaxProducers() throws Exception { @Test public void testGetSetDispatchRate() throws Exception { - DispatchRate dispatchRate = new DispatchRate(100, 10000, 1, true); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(100) + .dispatchThrottlingRateInByte(1000) + .ratePeriodInSecond(1) + .relativeToPublishRate(true) + .build(); log.info("Dispatch Rate: {} will set to the topic: {}", dispatchRate, testTopic); Awaitility.await() @@ -790,7 +835,12 @@ public void testGetSetDispatchRate() throws Exception { @Test public void testRemoveDispatchRate() throws Exception { - DispatchRate dispatchRate = new DispatchRate(100, 10000, 1, true); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(100) + .dispatchThrottlingRateInByte(1000) + .ratePeriodInSecond(1) + .relativeToPublishRate(true) + .build(); log.info("Dispatch Rate: {} will set to the topic: {}", dispatchRate, testTopic); Awaitility.await() @@ -817,20 +867,35 @@ public void testPolicyOverwrittenByNamespaceLevel() throws Exception { Awaitility.await() .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); - DispatchRate dispatchRate = new DispatchRate(200, 20000, 1, true); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(200) + .dispatchThrottlingRateInByte(20000) + .ratePeriodInSecond(1) + .relativeToPublishRate(true) + .build(); admin.namespaces().setDispatchRate(myNamespace, dispatchRate); Awaitility.await() - .untilAsserted(() -> Assert.assertEquals(admin.namespaces().getDispatchRate(myNamespace).dispatchThrottlingRateInMsg, 200)); + .untilAsserted(() -> Assert.assertEquals(admin.namespaces().getDispatchRate(myNamespace).getDispatchThrottlingRateInMsg(), 200)); - dispatchRate = new DispatchRate(100, 10000, 1, true); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(100) + .dispatchThrottlingRateInByte(10000) + .ratePeriodInSecond(1) + .relativeToPublishRate(true) + .build(); admin.topics().setDispatchRate(topic, dispatchRate); Awaitility.await() .untilAsserted(() -> Assert.assertNotNull(admin.topics().getDispatchRate(topic))); //1 Set ns level policy, topic level should not be overwritten - dispatchRate = new DispatchRate(300, 30000, 2, true); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(300) + .dispatchThrottlingRateInByte(30000) + .ratePeriodInSecond(2) + .relativeToPublishRate(true) + .build(); admin.namespaces().setDispatchRate(myNamespace, dispatchRate); Awaitility.await() @@ -901,8 +966,11 @@ public void testGetSetSubscriptionDispatchRate() throws Exception { Awaitility.await() .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); - DispatchRate dispatchRate = new DispatchRate(1000, - 1024 * 1024, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(1000) + .dispatchThrottlingRateInByte(1024 * 1024) + .ratePeriodInSecond(1) + .build(); log.info("Subscription Dispatch Rate: {} will set to the topic: {}", dispatchRate, topic); admin.topics().setSubscriptionDispatchRate(topic, dispatchRate); @@ -917,8 +985,8 @@ public void testGetSetSubscriptionDispatchRate() throws Exception { DispatchRateLimiter dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic) .get().get().getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); Assert.assertNotNull(dispatchRateLimiter); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.dispatchThrottlingRateInByte); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.dispatchThrottlingRateInMsg); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.getDispatchThrottlingRateInByte()); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.getDispatchThrottlingRateInMsg()); consumer.close(); admin.topics().delete(topic, true); @@ -932,8 +1000,11 @@ public void testGetSetSubscriptionDispatchRateAfterTopicLoaded() throws Exceptio Awaitility.await() .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); - DispatchRate dispatchRate = new DispatchRate(1000, - 1024 * 1024, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(1000) + .dispatchThrottlingRateInByte(1024 * 1024) + .ratePeriodInSecond(1) + .build(); log.info("Subscription Dispatch Rate: {} will set to the topic: {}", dispatchRate, topic); String subscriptionName = "test_subscription_rate"; @@ -948,8 +1019,8 @@ public void testGetSetSubscriptionDispatchRateAfterTopicLoaded() throws Exceptio DispatchRateLimiter dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic) .get().get().getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); Assert.assertNotNull(dispatchRateLimiter); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.dispatchThrottlingRateInByte); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.dispatchThrottlingRateInMsg); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.getDispatchThrottlingRateInByte()); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.getDispatchThrottlingRateInMsg()); consumer.close(); admin.topics().delete(topic, true); @@ -963,8 +1034,11 @@ public void testRemoveSubscriptionDispatchRate() throws Exception { Awaitility.await() .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); - DispatchRate dispatchRate = new DispatchRate(1000, - 1024 * 1024, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(1000) + .dispatchThrottlingRateInByte(1024 * 1024) + .ratePeriodInSecond(1) + .build(); log.info("Subscription Dispatch Rate: {} will set to the topic: {}", dispatchRate, topic); admin.topics().setSubscriptionDispatchRate(topic, dispatchRate); @@ -979,8 +1053,8 @@ public void testRemoveSubscriptionDispatchRate() throws Exception { DispatchRateLimiter dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic) .get().get().getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); Assert.assertNotNull(dispatchRateLimiter); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.dispatchThrottlingRateInByte); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.dispatchThrottlingRateInMsg); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.getDispatchThrottlingRateInByte()); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.getDispatchThrottlingRateInMsg()); // remove subscription dispatch rate admin.topics().removeSubscriptionDispatchRate(topic); @@ -990,8 +1064,8 @@ public void testRemoveSubscriptionDispatchRate() throws Exception { dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic) .get().get().getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); - Assert.assertNotEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.dispatchThrottlingRateInMsg); - Assert.assertNotEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.dispatchThrottlingRateInByte); + Assert.assertNotEquals(dispatchRateLimiter.getDispatchRateOnMsg(), dispatchRate.getDispatchThrottlingRateInByte()); + Assert.assertNotEquals(dispatchRateLimiter.getDispatchRateOnByte(), dispatchRate.getDispatchThrottlingRateInByte()); consumer.close(); admin.topics().delete(topic, true); @@ -1006,7 +1080,11 @@ public void testSubscriptionDispatchRatePolicyOverwrittenNamespaceLevel() throws .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); // set namespace level subscription dispatch rate - DispatchRate namespaceDispatchRate = new DispatchRate(100, 1024 * 1024, 1); + DispatchRate namespaceDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(100) + .dispatchThrottlingRateInByte(1024 * 1024) + .ratePeriodInSecond(1) + .build();; admin.namespaces().setSubscriptionDispatchRate(myNamespace, namespaceDispatchRate); Awaitility.await() @@ -1018,11 +1096,15 @@ public void testSubscriptionDispatchRatePolicyOverwrittenNamespaceLevel() throws // get subscription dispatch Rate limiter DispatchRateLimiter dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic) .get().get().getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), namespaceDispatchRate.dispatchThrottlingRateInMsg); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), namespaceDispatchRate.dispatchThrottlingRateInByte); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), namespaceDispatchRate.getDispatchThrottlingRateInMsg()); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), namespaceDispatchRate.getDispatchThrottlingRateInByte()); // set topic level subscription dispatch rate - DispatchRate topicDispatchRate = new DispatchRate(200, 2 * 1024 * 1024, 1); + DispatchRate topicDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(200) + .dispatchThrottlingRateInByte(2 * 1024 * 1024) + .ratePeriodInSecond(1) + .build();; admin.topics().setSubscriptionDispatchRate(topic, topicDispatchRate); Awaitility.await() @@ -1031,8 +1113,8 @@ public void testSubscriptionDispatchRatePolicyOverwrittenNamespaceLevel() throws // get subscription dispatch rate limiter dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic).get().get() .getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), topicDispatchRate.dispatchThrottlingRateInByte); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), topicDispatchRate.dispatchThrottlingRateInMsg); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), topicDispatchRate.getDispatchThrottlingRateInByte()); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), topicDispatchRate.getDispatchThrottlingRateInMsg()); // remove topic level subscription dispatch rate limiter admin.topics().removeSubscriptionDispatchRate(topic); @@ -1043,8 +1125,8 @@ public void testSubscriptionDispatchRatePolicyOverwrittenNamespaceLevel() throws // get subscription dispatch rate limiter dispatchRateLimiter = pulsar.getBrokerService().getTopicIfExists(topic).get().get() .getSubscription(subscriptionName).getDispatcher().getRateLimiter().get(); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), namespaceDispatchRate.dispatchThrottlingRateInByte); - Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), namespaceDispatchRate.dispatchThrottlingRateInMsg); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnByte(), namespaceDispatchRate.getDispatchThrottlingRateInByte()); + Assert.assertEquals(dispatchRateLimiter.getDispatchRateOnMsg(), namespaceDispatchRate.getDispatchThrottlingRateInMsg()); consumer.close(); admin.topics().delete(topic, true); @@ -1966,7 +2048,11 @@ public void testReplicatorRateApi() throws Exception { assertNull(admin.topics().getReplicatorDispatchRate(topic)); - DispatchRate dispatchRate = new DispatchRate(100,200L,10); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(100) + .dispatchThrottlingRateInByte(200) + .ratePeriodInSecond(10) + .build(); admin.topics().setReplicatorDispatchRate(topic, dispatchRate); Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getReplicatorDispatchRate(topic), dispatchRate)); @@ -1984,19 +2070,27 @@ public void testGetReplicatorRateApplied() throws Exception { .until(() -> pulsar.getTopicPoliciesService().cacheIsInitialized(TopicName.get(topic))); assertNull(admin.topics().getReplicatorDispatchRate(topic)); assertNull(admin.namespaces().getReplicatorDispatchRate(myNamespace)); - DispatchRate brokerDispatchRate = new DispatchRate( - pulsar.getConfiguration().getDispatchThrottlingRatePerReplicatorInMsg(), - pulsar.getConfiguration().getDispatchThrottlingRatePerReplicatorInByte(), - 1 - ); + DispatchRate brokerDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(pulsar.getConfiguration().getDispatchThrottlingRatePerReplicatorInMsg()) + .dispatchThrottlingRateInByte(pulsar.getConfiguration().getDispatchThrottlingRatePerReplicatorInByte()) + .ratePeriodInSecond(1) + .build(); assertEquals(admin.topics().getReplicatorDispatchRate(topic, true), brokerDispatchRate); - DispatchRate namespaceDispatchRate = new DispatchRate(10, 11, 12); + DispatchRate namespaceDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(11) + .ratePeriodInSecond(12) + .build(); admin.namespaces().setReplicatorDispatchRate(myNamespace, namespaceDispatchRate); Awaitility.await().untilAsserted(() -> assertNotNull(admin.namespaces().getReplicatorDispatchRate(myNamespace))); assertEquals(admin.topics().getReplicatorDispatchRate(topic, true), namespaceDispatchRate); - DispatchRate topicDispatchRate = new DispatchRate(20, 21, 22); + DispatchRate topicDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(20) + .dispatchThrottlingRateInByte(21) + .ratePeriodInSecond(22) + .build(); admin.topics().setReplicatorDispatchRate(topic, topicDispatchRate); Awaitility.await().untilAsserted(() -> assertNotNull(admin.topics().getReplicatorDispatchRate(topic))); assertEquals(admin.topics().getReplicatorDispatchRate(topic, true), topicDispatchRate); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java index ed60ef3f63a95..58ee083163e2d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java @@ -29,9 +29,13 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - +import com.google.common.collect.BoundType; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; import java.lang.reflect.Field; -import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; import java.util.HashMap; @@ -43,10 +47,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; - import javax.ws.rs.client.InvocationCallback; import javax.ws.rs.client.WebTarget; - +import lombok.Builder; +import lombok.Cleanup; +import lombok.Value; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -61,8 +67,8 @@ import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; import org.apache.pulsar.client.admin.internal.LookupImpl; -import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.client.admin.internal.TenantsImpl; +import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.Message; @@ -80,25 +86,26 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.BacklogQuota.RetentionPolicy; import org.apache.pulsar.common.policies.data.BrokerAssignment; import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationData; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; -import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; -import org.apache.pulsar.common.policies.data.PoliciesUtil; -import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.PoliciesUtil; import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.compaction.Compactor; @@ -111,16 +118,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.BoundType; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Range; -import com.google.common.collect.Sets; -import com.google.common.hash.Hashing; - -import lombok.Cleanup; -import lombok.extern.slf4j.Slf4j; - @Slf4j @Test(groups = "broker") public class V1_AdminApiTest extends MockedPulsarServiceBaseTest { @@ -164,7 +161,7 @@ public void setup() throws Exception { otheradmin = mockPulsarSetup.getAdmin(); // Setup namespaces - admin.clusters().createCluster("use", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/use/ns1"); @@ -201,27 +198,31 @@ public Object[][] topicTypeProvider() { @Test public void clusters() throws Exception { admin.clusters().createCluster("usw", - new ClusterDataImpl("http://broker.messaging.use.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.use.example.com:8080").build()); // "test" cluster is part of config-default cluster and it's znode gets created when PulsarService creates // failure-domain znode of this default cluster assertEquals(admin.clusters().getClusters(), Lists.newArrayList("use", "usw")); assertEquals(admin.clusters().getCluster("use"), - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.clusters().updateCluster("usw", - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080")); + ClusterData.builder().serviceUrl("http://new-broker.messaging.use.example.com:8080").build()); assertEquals(admin.clusters().getClusters(), Lists.newArrayList("use", "usw")); assertEquals(admin.clusters().getCluster("usw"), - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080")); + ClusterData.builder().serviceUrl("http://new-broker.messaging.use.example.com:8080").build()); admin.clusters().updateCluster("usw", - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080", - "https://new-broker.messaging.usw.example.com:4443")); + ClusterData.builder() + .serviceUrl("http://new-broker.messaging.usw.example.com:8080") + .serviceUrlTls("https://new-broker.messaging.usw.example.com:4443") + .build()); assertEquals(admin.clusters().getClusters(), Lists.newArrayList("use", "usw")); assertEquals(admin.clusters().getCluster("usw"), - new ClusterDataImpl("http://new-broker.messaging.usw.example.com:8080", - "https://new-broker.messaging.usw.example.com:4443")); + ClusterData.builder() + .serviceUrl("http://new-broker.messaging.usw.example.com:8080") + .serviceUrlTls("https://new-broker.messaging.usw.example.com:4443") + .build()); admin.clusters().deleteCluster("usw"); Thread.sleep(300); @@ -234,7 +235,7 @@ public void clusters() throws Exception { // Check name validation try { - admin.clusters().createCluster("bf!", new ClusterDataImpl("http://dummy.messaging.example.com")); + admin.clusters().createCluster("bf!", ClusterData.builder().serviceUrl("http://dummy.messaging.example.com").build()); fail("should have failed"); } catch (PulsarAdminException e) { assertTrue(e instanceof PreconditionFailedException); @@ -246,35 +247,36 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { try { // create String policyName1 = "policy-1"; - NamespaceIsolationDataImpl nsPolicyData1 = new NamespaceIsolationDataImpl(); - nsPolicyData1.namespaces = new ArrayList<>(); - nsPolicyData1.namespaces.add("other/use/other.*"); - nsPolicyData1.primary = new ArrayList<>(); - // match all broker. make it easy to verify `getBrokersWithNamespaceIsolationPolicy` later - nsPolicyData1.primary.add(".*"); - - nsPolicyData1.secondary = new ArrayList<>(); - nsPolicyData1.secondary.add("prod1-broker.*.messaging.use.example.com"); - nsPolicyData1.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData1.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData1.auto_failover_policy.parameters = new HashMap<>(); - nsPolicyData1.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData1.auto_failover_policy.parameters.put("usage_threshold", "100"); + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData1 = NamespaceIsolationData.builder() + .namespaces(Lists.newArrayList("other/use/other.*")) + .primary(Lists.newArrayList(".*")) // match all broker. make it easy to verify `getBrokersWithNamespaceIsolationPolicy` later + .secondary(Lists.newArrayList("prod1-broker.*.messaging.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); + admin.clusters().createNamespaceIsolationPolicy("use", policyName1, nsPolicyData1); String policyName2 = "policy-2"; - NamespaceIsolationDataImpl nsPolicyData2 = new NamespaceIsolationDataImpl(); - nsPolicyData2.namespaces = new ArrayList<>(); - nsPolicyData2.namespaces.add("other/use/other.*"); - nsPolicyData2.primary = new ArrayList<>(); - nsPolicyData2.primary.add("prod1-broker[4-6].messaging.use.example.com"); - nsPolicyData2.secondary = new ArrayList<>(); - nsPolicyData2.secondary.add("prod1-broker.*.messaging.use.example.com"); - nsPolicyData2.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData2.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData2.auto_failover_policy.parameters = new HashMap<>(); - nsPolicyData2.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData2.auto_failover_policy.parameters.put("usage_threshold", "100"); + Map parameters2 = new HashMap<>(); + parameters2.put("min_limit", "1"); + parameters2.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData2 = NamespaceIsolationData.builder() + .namespaces(Lists.newArrayList("other/use/other.*")) + .primary(Lists.newArrayList("prod1-broker[4-6].messaging.use.example.com")) + .secondary(Lists.newArrayList("prod1-broker.*.messaging.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); admin.clusters().createNamespaceIsolationPolicy("use", policyName2, nsPolicyData2); // verify create indirectly with get @@ -288,8 +290,8 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertTrue(isoList.get(0).isPrimary()); // verify update of primary - nsPolicyData1.primary.remove(0); - nsPolicyData1.primary.add("prod1-broker[1-2].messaging.use.example.com"); + nsPolicyData1.getPrimary().remove(0); + nsPolicyData1.getPrimary().add("prod1-broker[1-2].messaging.use.example.com"); admin.clusters().updateNamespaceIsolationPolicy("use", policyName1, nsPolicyData1); // verify primary change @@ -297,8 +299,8 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName1), nsPolicyData1); // verify update of secondary - nsPolicyData1.secondary.remove(0); - nsPolicyData1.secondary.add("prod1-broker[3-4].messaging.use.example.com"); + nsPolicyData1.getSecondary().remove(0); + nsPolicyData1.getSecondary().add("prod1-broker[3-4].messaging.use.example.com"); admin.clusters().updateNamespaceIsolationPolicy("use", policyName1, nsPolicyData1); // verify secondary change @@ -306,7 +308,7 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName1), nsPolicyData1); // verify update of failover policy limit - nsPolicyData1.auto_failover_policy.parameters.put("min_limit", "10"); + nsPolicyData1.getAutoFailoverPolicy().getParameters().put("min_limit", "10"); admin.clusters().updateNamespaceIsolationPolicy("use", policyName1, nsPolicyData1); // verify min_limit change @@ -314,7 +316,7 @@ public void clusterNamespaceIsolationPolicies() throws PulsarAdminException { assertEquals(policiesMap.get(policyName1), nsPolicyData1); // verify update of failover usage_threshold limit - nsPolicyData1.auto_failover_policy.parameters.put("usage_threshold", "80"); + nsPolicyData1.getAutoFailoverPolicy().getParameters().put("usage_threshold", "80"); admin.clusters().updateNamespaceIsolationPolicy("use", policyName1, nsPolicyData1); // verify usage_threshold change @@ -603,7 +605,7 @@ public void properties() throws PulsarAdminException { @Test public void namespaces() throws Exception { - admin.clusters().createCluster("usw", new ClusterDataImpl()); + admin.clusters().createCluster("usw", ClusterData.builder().build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use", "usw")); admin.tenants().updateTenant("prop-xyz", tenantInfo); @@ -613,8 +615,8 @@ public void namespaces() throws Exception { admin.namespaces().createNamespace("prop-xyz/use/ns2"); admin.namespaces().createNamespace("prop-xyz/use/ns3", 4); - assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns3").bundles.numBundles, 4); - assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns3").bundles.boundaries.size(), 5); + assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns3").bundles.getNumBundles(), 4); + assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns3").bundles.getBoundaries().size(), 5); admin.namespaces().deleteNamespace("prop-xyz/use/ns3"); @@ -643,15 +645,15 @@ public void namespaces() throws Exception { Policies policies = new Policies(); policies.bundles = PoliciesUtil.defaultBundle(); - policies.auth_policies.namespace_auth.put("my-role", EnumSet.allOf(AuthAction.class)); + policies.auth_policies.getNamespaceAuthentication().put("my-role", EnumSet.allOf(AuthAction.class)); assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns1"), policies); - assertEquals(admin.namespaces().getPermissions("prop-xyz/use/ns1"), policies.auth_policies.namespace_auth); + assertEquals(admin.namespaces().getPermissions("prop-xyz/use/ns1"), policies.auth_policies.getNamespaceAuthentication()); assertEquals(admin.namespaces().getTopics("prop-xyz/use/ns1"), Lists.newArrayList()); admin.namespaces().revokePermissionsOnNamespace("prop-xyz/use/ns1", "my-role"); - policies.auth_policies.namespace_auth.remove("my-role"); + policies.auth_policies.getNamespaceAuthentication().remove("my-role"); assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns1"), policies); assertNull(admin.namespaces().getPersistence("prop-xyz/use/ns1")); @@ -724,10 +726,10 @@ public void persistentTopics(String topicName) throws Exception { publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/" + topicName, 10); TopicStats topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); - assertEquals(topicStats.subscriptions.get("my-sub").consumers.size(), 1); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 10); - assertEquals(topicStats.publishers.size(), 0); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); + assertEquals(topicStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 10); + assertEquals(topicStats.getPublishers().size(), 0); PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(persistentTopicName, false); assertEquals(internalStats.cursors.keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); @@ -748,11 +750,11 @@ public void persistentTopics(String topicName) throws Exception { admin.topics().skipMessages(persistentTopicName, "my-sub", 5); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 5); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 5); admin.topics().skipAllMessages(persistentTopicName, "my-sub"); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 0); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 0); consumer.close(); client.close(); @@ -761,8 +763,8 @@ public void persistentTopics(String topicName) throws Exception { assertEquals(admin.topics().getSubscriptions(persistentTopicName), Lists.newArrayList()); topicStats = admin.topics().getStats(persistentTopicName); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet()); - assertEquals(topicStats.publishers.size(), 0); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet()); + assertEquals(topicStats.getPublishers().size(), 0); try { admin.topics().skipAllMessages(persistentTopicName, "my-sub"); @@ -849,22 +851,22 @@ public void partitionedTopics(String topicName) throws Exception { // test cumulative stats for partitioned topic PartitionedTopicStats topicStats = admin.topics().getPartitionedStats(partitionedTopicName, false); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); - assertEquals(topicStats.subscriptions.get("my-sub").consumers.size(), 1); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 10); - assertEquals(topicStats.publishers.size(), 1); - assertEquals(topicStats.partitions, Maps.newHashMap()); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); + assertEquals(topicStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 10); + assertEquals(topicStats.getPublishers().size(), 1); + assertEquals(topicStats.getPartitions(), Maps.newHashMap()); // test per partition stats for partitioned topic topicStats = admin.topics().getPartitionedStats(partitionedTopicName, true); - assertEquals(topicStats.metadata.partitions, 4); - assertEquals(topicStats.partitions.keySet(), + assertEquals(topicStats.getMetadata().partitions, 4); + assertEquals(topicStats.getPartitions().keySet(), Sets.newHashSet(partitionedTopicName + "-partition-0", partitionedTopicName + "-partition-1", partitionedTopicName + "-partition-2", partitionedTopicName + "-partition-3")); - TopicStats partitionStats = topicStats.partitions.get(partitionedTopicName + "-partition-0"); - assertEquals(partitionStats.publishers.size(), 1); - assertEquals(partitionStats.subscriptions.get("my-sub").consumers.size(), 1); - assertEquals(partitionStats.subscriptions.get("my-sub").msgBacklog, 3, 1); + TopicStats partitionStats = topicStats.getPartitions().get(partitionedTopicName + "-partition-0"); + assertEquals(partitionStats.getPublishers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get("my-sub").getMsgBacklog(), 3, 1); try { admin.topics().skipMessages(partitionedTopicName, "my-sub", 5); @@ -875,7 +877,7 @@ public void partitionedTopics(String topicName) throws Exception { admin.topics().skipAllMessages(partitionedTopicName, "my-sub"); topicStats = admin.topics().getPartitionedStats(partitionedTopicName, false); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 0); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 0); producer.close(); consumer.close(); @@ -1255,26 +1257,26 @@ public void testClearBacklogOnNamespace(Integer numBundles) throws Exception { admin.namespaces().clearNamespaceBacklogForSubscription("prop-xyz/use/ns1-bundles", "my-sub"); - long backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds2").subscriptions - .get("my-sub").msgBacklog; + long backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds2").getSubscriptions() + .get("my-sub").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds1").subscriptions - .get("my-sub").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds1").getSubscriptions() + .get("my-sub").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds1").subscriptions - .get("my-sub-1").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds1").getSubscriptions() + .get("my-sub-1").getMsgBacklog(); assertEquals(backlog, 10); admin.namespaces().clearNamespaceBacklog("prop-xyz/use/ns1-bundles"); - backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds1").subscriptions - .get("my-sub-1").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds1").getSubscriptions() + .get("my-sub-1").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds2").subscriptions - .get("my-sub-1").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds2").getSubscriptions() + .get("my-sub-1").getMsgBacklog(); assertEquals(backlog, 0); - backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds2").subscriptions - .get("my-sub-2").msgBacklog; + backlog = admin.topics().getStats("persistent://prop-xyz/use/ns1-bundles/ds2").getSubscriptions() + .get("my-sub-2").getMsgBacklog(); assertEquals(backlog, 0); } @@ -1359,11 +1361,16 @@ public void backlogQuotas() throws Exception { assertNull(quotaMap.get(BacklogQuotaType.destination_storage)); admin.namespaces().setBacklogQuota("prop-xyz/use/ns1", - new BacklogQuota(1 * 1024 * 1024, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(1 * 1024 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); quotaMap = admin.namespaces().getBacklogQuotaMap("prop-xyz/use/ns1"); assertEquals(quotaMap.size(), 1); - assertEquals(quotaMap.get(BacklogQuotaType.destination_storage), - new BacklogQuota(1 * 1024 * 1024, RetentionPolicy.producer_exception)); + assertEquals(quotaMap.get(BacklogQuotaType.destination_storage), BacklogQuota.builder() + .limitSize(1 * 1024 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); admin.namespaces().removeBacklogQuota("prop-xyz/use/ns1"); @@ -1686,19 +1693,25 @@ public void persistentTopicsInvalidCursorReset() throws Exception { admin.topics().delete(topicName); } + @Value + @Builder + static class CustomTenantAdmin implements TenantInfo { + private final int newTenant; + private final Set adminRoles; + private final Set allowedClusters; + } + @Test public void testObjectWithUnknownProperties() { - - class CustomPropertyAdmin extends TenantInfoImpl { - @SuppressWarnings("unused") - public int newProperty; - } - - TenantInfoImpl pa = new TenantInfoImpl(Sets.newHashSet("test_appid1", "test_appid2"), Sets.newHashSet("use")); - CustomPropertyAdmin cpa = new CustomPropertyAdmin(); - cpa.setAdminRoles(pa.getAdminRoles()); - cpa.setAllowedClusters(pa.getAllowedClusters()); - cpa.newProperty = 100; + TenantInfo pa = TenantInfo.builder() + .adminRoles(Sets.newHashSet("test_appid1", "test_appid2")) + .allowedClusters(Sets.newHashSet("use")) + .build(); + CustomTenantAdmin cpa = CustomTenantAdmin.builder() + .adminRoles(pa.getAdminRoles()) + .allowedClusters(pa.getAllowedClusters()) + .newTenant(100) + .build(); try { admin.tenants().createTenant("test-property", cpa); @@ -1743,18 +1756,18 @@ public void testPersistentTopicsExpireMessages() throws Exception { publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/ds2", 10); TopicStats topicStats = admin.topics().getStats("persistent://prop-xyz/use/ns1/ds2"); - assertEquals(topicStats.subscriptions.get("my-sub1").msgBacklog, 10); - assertEquals(topicStats.subscriptions.get("my-sub2").msgBacklog, 10); - assertEquals(topicStats.subscriptions.get("my-sub3").msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get("my-sub1").getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get("my-sub2").getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get("my-sub3").getMsgBacklog(), 10); Thread.sleep(1000); // wait for 1 seconds to expire message admin.topics().expireMessages("persistent://prop-xyz/use/ns1/ds2", "my-sub1", 1); Thread.sleep(1000); // wait for 1 seconds to execute expire message as it is async topicStats = admin.topics().getStats("persistent://prop-xyz/use/ns1/ds2"); - assertEquals(topicStats.subscriptions.get("my-sub1").msgBacklog, 0); - assertEquals(topicStats.subscriptions.get("my-sub2").msgBacklog, 10); - assertEquals(topicStats.subscriptions.get("my-sub3").msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get("my-sub1").getMsgBacklog(), 0); + assertEquals(topicStats.getSubscriptions().get("my-sub2").getMsgBacklog(), 10); + assertEquals(topicStats.getSubscriptions().get("my-sub3").getMsgBacklog(), 10); try { admin.topics().expireMessagesForAllSubscriptions("persistent://prop-xyz/use/ns1/ds2", 1); @@ -1765,9 +1778,9 @@ public void testPersistentTopicsExpireMessages() throws Exception { Thread.sleep(1000); // wait for 1 seconds to execute expire message as it is async topicStats = admin.topics().getStats("persistent://prop-xyz/use/ns1/ds2"); - assertEquals(topicStats.subscriptions.get("my-sub1").msgBacklog, 0); - assertEquals(topicStats.subscriptions.get("my-sub2").msgBacklog, 0); - assertEquals(topicStats.subscriptions.get("my-sub3").msgBacklog, 0); + assertEquals(topicStats.getSubscriptions().get("my-sub1").getMsgBacklog(), 0); + assertEquals(topicStats.getSubscriptions().get("my-sub2").getMsgBacklog(), 0); + assertEquals(topicStats.getSubscriptions().get("my-sub3").getMsgBacklog(), 0); consumer1.close(); consumer2.close(); @@ -1806,24 +1819,24 @@ public void testPersistentTopicExpireMessageOnPartitionTopic() throws Exception PartitionedTopicStats topicStats = admin.topics() .getPartitionedStats("persistent://prop-xyz/use/ns1/ds1", true); - assertEquals(topicStats.subscriptions.get("my-sub").msgBacklog, 10); + assertEquals(topicStats.getSubscriptions().get("my-sub").getMsgBacklog(), 10); - TopicStats partitionStatsPartition0 = topicStats.partitions + TopicStats partitionStatsPartition0 = topicStats.getPartitions() .get("persistent://prop-xyz/use/ns1/ds1-partition-0"); - TopicStats partitionStatsPartition1 = topicStats.partitions + TopicStats partitionStatsPartition1 = topicStats.getPartitions() .get("persistent://prop-xyz/use/ns1/ds1-partition-1"); - assertEquals(partitionStatsPartition0.subscriptions.get("my-sub").msgBacklog, 3, 1); - assertEquals(partitionStatsPartition1.subscriptions.get("my-sub").msgBacklog, 3, 1); + assertEquals(partitionStatsPartition0.getSubscriptions().get("my-sub").getMsgBacklog(), 3, 1); + assertEquals(partitionStatsPartition1.getSubscriptions().get("my-sub").getMsgBacklog(), 3, 1); Thread.sleep(1000); admin.topics().expireMessagesForAllSubscriptions("persistent://prop-xyz/use/ns1/ds1", 1); Thread.sleep(1000); topicStats = admin.topics().getPartitionedStats("persistent://prop-xyz/use/ns1/ds1", true); - partitionStatsPartition0 = topicStats.partitions.get("persistent://prop-xyz/use/ns1/ds1-partition-0"); - partitionStatsPartition1 = topicStats.partitions.get("persistent://prop-xyz/use/ns1/ds1-partition-1"); - assertEquals(partitionStatsPartition0.subscriptions.get("my-sub").msgBacklog, 0); - assertEquals(partitionStatsPartition1.subscriptions.get("my-sub").msgBacklog, 0); + partitionStatsPartition0 = topicStats.getPartitions().get("persistent://prop-xyz/use/ns1/ds1-partition-0"); + partitionStatsPartition1 = topicStats.getPartitions().get("persistent://prop-xyz/use/ns1/ds1-partition-1"); + assertEquals(partitionStatsPartition0.getSubscriptions().get("my-sub").getMsgBacklog(), 0); + assertEquals(partitionStatsPartition1.getSubscriptions().get("my-sub").getMsgBacklog(), 0); producer.close(); consumer.close(); @@ -1930,8 +1943,8 @@ public void failed(Throwable e) { uriStats.completeExceptionally(e); } }); - assertEquals(urlStats.get().subscriptions.size(), 1); - assertEquals(uriStats.get().subscriptions.size(), 1); + assertEquals(urlStats.get().getSubscriptions().size(), 1); + assertEquals(uriStats.get().getSubscriptions().size(), 1); } static class MockedPulsarService extends MockedPulsarServiceBaseTest { @@ -1965,12 +1978,12 @@ public PulsarAdmin getAdmin() { @Test public void testTopicBundleRangeLookup() throws Exception { - admin.clusters().createCluster("usw", new ClusterDataImpl()); + admin.clusters().createCluster("usw", ClusterData.builder().build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use", "usw")); admin.tenants().updateTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/use/getBundleNs", 100); - assertEquals(admin.namespaces().getPolicies("prop-xyz/use/getBundleNs").bundles.numBundles, 100); + assertEquals(admin.namespaces().getPolicies("prop-xyz/use/getBundleNs").bundles.getNumBundles(), 100); // (1) create a topic final String topicName = "persistent://prop-xyz/use/getBundleNs/topic1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest2.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest2.java index 71f64b4307f0f..607cdb6eb8180 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest2.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest2.java @@ -31,6 +31,7 @@ import com.google.common.collect.Sets; import java.net.URL; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -59,6 +60,7 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.FailureDomainImpl; @@ -94,7 +96,7 @@ public void setup() throws Exception { mockPulsarSetup.setup(); // Setup namespaces - admin.clusters().createCluster("use", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use")); admin.tenants().createTenant("prop-xyz", tenantInfo); admin.namespaces().createNamespace("prop-xyz/use/ns1"); @@ -193,26 +195,26 @@ public void testIncrementPartitionsOfTopic() throws Exception { // test cumulative stats for partitioned topic PartitionedTopicStats topicStats = admin.topics().getPartitionedStats(partitionedTopicName, false); - assertEquals(topicStats.subscriptions.keySet(), Sets.newTreeSet(Lists.newArrayList(subName1, subName2))); - assertEquals(topicStats.subscriptions.get(subName2).consumers.size(), 1); - assertEquals(topicStats.subscriptions.get(subName2).msgBacklog, totalMessages); - assertEquals(topicStats.publishers.size(), 1); - assertEquals(topicStats.partitions, Maps.newHashMap()); + assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList(subName1, subName2))); + assertEquals(topicStats.getSubscriptions().get(subName2).getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get(subName2).getMsgBacklog(), totalMessages); + assertEquals(topicStats.getPublishers().size(), 1); + assertEquals(topicStats.getPartitions(), Maps.newHashMap()); // (5) verify: each partition should have backlog topicStats = admin.topics().getPartitionedStats(partitionedTopicName, true); - assertEquals(topicStats.metadata.partitions, newPartitions); + assertEquals(topicStats.getMetadata().partitions, newPartitions); Set partitionSet = Sets.newHashSet(); for (int i = 0; i < newPartitions; i++) { partitionSet.add(partitionedTopicName + "-partition-" + i); } - assertEquals(topicStats.partitions.keySet(), partitionSet); + assertEquals(topicStats.getPartitions().keySet(), partitionSet); for (int i = 0; i < newPartitions; i++) { - TopicStats partitionStats = topicStats.partitions + TopicStats partitionStats = topicStats.getPartitions() .get(TopicName.get(partitionedTopicName).getPartition(i).toString()); - assertEquals(partitionStats.publishers.size(), 1); - assertEquals(partitionStats.subscriptions.get(subName2).consumers.size(), 1); - assertEquals(partitionStats.subscriptions.get(subName2).msgBacklog, 2, 1); + assertEquals(partitionStats.getPublishers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get(subName2).getConsumers().size(), 1); + assertEquals(partitionStats.getSubscriptions().get(subName2).getMsgBacklog(), 2, 1); } producer.close(); @@ -247,7 +249,7 @@ public void nonPersistentTopics() throws Exception { NonPersistentTopicStats topicStats = admin.nonPersistentTopics().getStats(persistentTopicName); assertEquals(topicStats.getSubscriptions().keySet(), Sets.newTreeSet(Lists.newArrayList("my-sub"))); - assertEquals(topicStats.getSubscriptions().get("my-sub").consumers.size(), 1); + assertEquals(topicStats.getSubscriptions().get("my-sub").getConsumers().size(), 1); assertEquals(topicStats.getPublishers().size(), 0); PersistentTopicInternalStats internalStats = admin.nonPersistentTopics().getInternalStats(persistentTopicName); @@ -550,13 +552,13 @@ public void testLoadReportApi() throws Exception { @Test public void testPeerCluster() throws Exception { admin.clusters().createCluster("us-west1", - new ClusterDataImpl("http://broker.messaging.west1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west1.example.com:8080").build()); admin.clusters().createCluster("us-west2", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-east1", - new ClusterDataImpl("http://broker.messaging.east1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.east1.example.com:8080").build()); admin.clusters().createCluster("us-east2", - new ClusterDataImpl("http://broker.messaging.east2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.east2.example.com:8080").build()); admin.clusters().updatePeerClusterNames("us-west1", Sets.newLinkedHashSet(Lists.newArrayList("us-west2"))); assertEquals(admin.clusters().getCluster("us-west1").getPeerClusterNames(), Lists.newArrayList("us-west2")); @@ -595,18 +597,18 @@ public void testPeerCluster() throws Exception { @Test public void testReplicationPeerCluster() throws Exception { admin.clusters().createCluster("us-west1", - new ClusterDataImpl("http://broker.messaging.west1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west1.example.com:8080").build()); admin.clusters().createCluster("us-west2", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-west3", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-west4", - new ClusterDataImpl("http://broker.messaging.west2.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.west2.example.com:8080").build()); admin.clusters().createCluster("us-east1", - new ClusterDataImpl("http://broker.messaging.east1.example.com:8080")); + ClusterData.builder().serviceUrl("http://broker.messaging.east1.example.com:8080").build()); admin.clusters().createCluster("us-east2", - new ClusterDataImpl("http://broker.messaging.east2.example.com:8080")); - admin.clusters().createCluster("global", new ClusterDataImpl()); + ClusterData.builder().serviceUrl("http://broker.messaging.east2.example.com:8080").build()); + admin.clusters().createCluster("global", ClusterData.builder().build()); final String property = "peer-prop"; Set allowedClusters = Sets.newHashSet("us-west1", "us-west2", "us-west3", "us-west4", "us-east1", @@ -649,10 +651,14 @@ public void clusterFailureDomain() throws PulsarAdminException { final String cluster = pulsar.getConfiguration().getClusterName(); admin.clusters().createCluster(cluster, - new ClusterDataImpl(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls())); + ClusterData.builder() + .serviceUrl(pulsar.getSafeWebServiceAddress()) + .serviceUrlTls(pulsar.getWebServiceAddressTls()) + .build()); // create - FailureDomainImpl domain = new FailureDomainImpl(); - domain.setBrokers(Sets.newHashSet("b1", "b2", "b3")); + FailureDomain domain = FailureDomain.builder() + .brokers(Sets.newHashSet("b1", "b2", "b3")) + .build(); admin.clusters().createFailureDomain(cluster, "domain-1", domain); admin.clusters().updateFailureDomain(cluster, "domain-1", domain); @@ -761,25 +767,25 @@ public void testPublishConsumerStats() throws Exception { TopicStats stats; try { stats = admin.topics().getStats(topic); - return stats.publishers.size() > 0 && stats.subscriptions.get(subscriberName) != null - && stats.subscriptions.get(subscriberName).consumers.size() > 0; + return stats.getPublishers().size() > 0 && stats.getSubscriptions().get(subscriberName) != null + && stats.getSubscriptions().get(subscriberName).getConsumers().size() > 0; } catch (PulsarAdminException e) { return false; } }, 5, 200); TopicStats topicStats = admin.topics().getStats(topic); - assertEquals(topicStats.publishers.size(), 1); - assertNotNull(topicStats.publishers.get(0).getAddress()); - assertNotNull(topicStats.publishers.get(0).getClientVersion()); - assertNotNull(topicStats.publishers.get(0).getConnectedSince()); - assertNotNull(topicStats.publishers.get(0).getProducerName()); - assertEquals(topicStats.publishers.get(0).getProducerName(), producerName); - - SubscriptionStats subscriber = topicStats.subscriptions.get(subscriberName); + assertEquals(topicStats.getPublishers().size(), 1); + assertNotNull(topicStats.getPublishers().get(0).getAddress()); + assertNotNull(topicStats.getPublishers().get(0).getClientVersion()); + assertNotNull(topicStats.getPublishers().get(0).getConnectedSince()); + assertNotNull(topicStats.getPublishers().get(0).getProducerName()); + assertEquals(topicStats.getPublishers().get(0).getProducerName(), producerName); + + SubscriptionStats subscriber = topicStats.getSubscriptions().get(subscriberName); assertNotNull(subscriber); - assertEquals(subscriber.consumers.size(), 1); - ConsumerStats consumerStats = subscriber.consumers.get(0); + assertEquals(subscriber.getConsumers().size(), 1); + ConsumerStats consumerStats = subscriber.getConsumers().get(0); assertNotNull(consumerStats.getAddress()); assertNotNull(consumerStats.getClientVersion()); assertNotNull(consumerStats.getConnectedSince()); @@ -804,7 +810,7 @@ public void testTenantNameWithUnderscore() throws Exception { .create(); TopicStats stats = admin.topics().getStats(topic); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); producer.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index ee68cfd4eb458..45b11dea4e90e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -36,6 +36,7 @@ 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.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -72,7 +73,7 @@ protected void setup() throws Exception { conf.setSystemTopicEnabled(true); conf.setTransactionBufferSnapshotMaxTransactionCount(1); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("pulsar", tenantInfo); admin.namespaces().createNamespace("pulsar/system", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java index d128e957fcd58..2b6c2420db347 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java @@ -20,6 +20,7 @@ import static org.testng.Assert.fail; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.annotations.AfterClass; @@ -69,7 +70,7 @@ public void setup() throws Exception { try (PulsarAdmin admin = PulsarAdmin.builder() .authentication(new MockAuthentication("pass.pass")) .serviceHttpUrl(brokerUrl.toString()).build()) { - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("public", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("public/default"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java index 5e5e7421064fa..dcdc602985b64 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.admin.PulsarAdminBuilder; 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.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; @@ -74,7 +75,7 @@ public void simple() throws Exception { assertFalse(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); - admin.clusters().createCluster("c1", new ClusterDataImpl()); + admin.clusters().createCluster("c1", ClusterData.builder().build()); admin.tenants().createTenant("p1", new TenantInfoImpl(Sets.newHashSet("role1"), Sets.newHashSet("c1"))); waitForChange(); admin.namespaces().createNamespace("p1/c1/ns1"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthorizationProvider.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthorizationProvider.java index f61b0de5b4bc3..421da4fec81a2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthorizationProvider.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthorizationProvider.java @@ -31,7 +31,7 @@ import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.policies.data.TopicOperation; @@ -58,7 +58,7 @@ public CompletableFuture isSuperUser(String role, ServiceConfiguration } @Override - public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfoImpl tenantInfo, + public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfo tenantInfo, AuthenticationDataSource authenticationData) { return roleAuthorizedAsync(role); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index ac5f104ec1b64..d265fb8a5c922 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -54,6 +54,7 @@ import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -313,7 +314,7 @@ protected void waitForZooKeeperWatchers() { protected TenantInfoImpl createDefaultTenantInfo() throws PulsarAdminException { // create local cluster if not exist if (!admin.clusters().getClusters().contains(configClusterName)) { - admin.clusters().createCluster(configClusterName, new ClusterDataImpl()); + admin.clusters().createCluster(configClusterName, ClusterData.builder().build()); } Set allowedClusters = Sets.newHashSet(); allowedClusters.add(configClusterName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java index f125984cb0f66..cca562db4e0c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java @@ -31,6 +31,7 @@ import java.lang.reflect.Field; import java.net.URL; +import java.util.Collections; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -55,7 +56,9 @@ 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.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -168,7 +171,7 @@ void shutdown() throws Exception { private void createCluster(ZooKeeper zk, ServiceConfiguration config) throws Exception { ZkUtils.createFullPathOptimistic(zk, "/admin/clusters/" + config.getClusterName(), ObjectMapperFactory.getThreadLocal().writeValueAsBytes( - new ClusterDataImpl("http://" + config.getAdvertisedAddress() + ":" + config.getWebServicePort().get())), + ClusterData.builder().serviceUrl("http://" + config.getAdvertisedAddress() + ":" + config.getWebServicePort().get()).build()), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } @@ -402,12 +405,14 @@ public void testBrokerSelectionForAntiAffinityGroup() throws Exception { final String namespace2 = tenant + "/" + cluster + "/ns2"; final String namespaceAntiAffinityGroup = "group"; - FailureDomainImpl domain1 = new FailureDomainImpl(); - domain1.brokers = Sets.newHashSet(broker1); + FailureDomain domain1 = FailureDomain.builder() + .brokers(Collections.singleton(broker1)) + .build(); admin1.clusters().createFailureDomain(cluster, "domain1", domain1); - FailureDomainImpl domain2 = new FailureDomainImpl(); - domain2.brokers = Sets.newHashSet(broker2); + FailureDomain domain2 = FailureDomain.builder() + .brokers(Collections.singleton(broker2)) + .build(); admin1.clusters().createFailureDomain(cluster, "domain2", domain2); admin1.tenants().createTenant(tenant, new TenantInfoImpl(null, Sets.newHashSet(cluster))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java index b1fd3682bf78b..faf5125482b94 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java @@ -33,6 +33,7 @@ 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.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.functions.worker.WorkerService; @@ -87,7 +88,7 @@ public void anErrorShouldBeThrowBeforeLeaderElected() throws PulsarServerExcepti final String namespace = "ns"; @Cleanup PulsarAdmin adminClient = PulsarAdmin.builder().serviceHttpUrl(pulsar.getWebServiceAddress()).build(); - adminClient.clusters().createCluster(clusterName, new ClusterDataImpl(pulsar.getWebServiceAddress())); + adminClient.clusters().createCluster(clusterName, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); adminClient.tenants().createTenant(tenant, new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet(clusterName))); adminClient.namespaces().createNamespace(tenant + "/" + namespace, 16); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index d48c2caeedd0b..175a1eb90fac4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -32,6 +32,7 @@ import java.lang.reflect.Method; import java.net.URL; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,9 +52,11 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.internal.NamespacesImpl; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.ResourceQuota; @@ -561,7 +564,10 @@ private BundlesData getBundles(int numBundles) { } curPartition += segSize; } - return new BundlesData(partitions); + return BundlesData.builder() + .boundaries(partitions) + .numBundles(partitions.size() - 1) + .build(); } private void createNamespace(PulsarService pulsar, String namespace, int numBundles) throws Exception { @@ -708,53 +714,53 @@ private void createNamespacePolicies(PulsarService pulsar) throws Exception { NamespaceIsolationPolicies policies = new NamespaceIsolationPolicies(); // set up policy that use this broker as primary - NamespaceIsolationDataImpl policyData = new NamespaceIsolationDataImpl(); - policyData.namespaces = new ArrayList(); - policyData.namespaces.add("pulsar/use/primary-ns.*"); - policyData.primary = new ArrayList(); + Map parameters = new HashMap<>(); + parameters.put("min_limit", "1"); + parameters.put("usage_threshold", "100"); + + List allBrokers = new ArrayList<>(); for (int i = 0; i < BROKER_COUNT; i++) { - policyData.primary.add(pulsarServices[i].getAdvertisedAddress()); + allBrokers.add(pulsarServices[i].getAdvertisedAddress()); } - policyData.secondary = new ArrayList(); - policyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - policyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - policyData.auto_failover_policy.parameters = new HashMap<>(); - policyData.auto_failover_policy.parameters.put("min_limit", "1"); - policyData.auto_failover_policy.parameters.put("usage_threshold", "100"); + + NamespaceIsolationData policyData = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("pulsar/use/primary-ns.*")) + .primary(allBrokers) + .secondary(Collections.emptyList()) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build()) + .build(); policies.setPolicy("primaryBrokerPolicy", policyData); - // set up policy that use this broker as secondary - policyData = new NamespaceIsolationDataImpl(); - policyData.namespaces = new ArrayList(); - policyData.namespaces.add("pulsar/use/secondary-ns.*"); - policyData.primary = new ArrayList(); - policyData.primary.add(pulsarServices[0].getAdvertisedAddress()); - policyData.secondary = new ArrayList(); + List allExceptFirstBroker = new ArrayList<>(); for (int i = 1; i < BROKER_COUNT; i++) { - policyData.secondary.add(pulsarServices[i].getAdvertisedAddress()); + allExceptFirstBroker.add(pulsarServices[i].getAdvertisedAddress()); } - policyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - policyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - policyData.auto_failover_policy.parameters = new HashMap(); - policyData.auto_failover_policy.parameters.put("min_limit", "1"); - policyData.auto_failover_policy.parameters.put("usage_threshold", "100"); + + // set up policy that use this broker as secondary + policyData = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("pulsar/use/secondary-ns.*")) + .primary(Collections.singletonList(pulsarServices[0].getWebServiceAddress())) + .secondary(allExceptFirstBroker) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build()) + .build(); policies.setPolicy("secondaryBrokerPolicy", policyData); // set up policy that do not use this broker (neither primary nor secondary) - policyData = new NamespaceIsolationDataImpl(); - policyData.namespaces = new ArrayList(); - policyData.namespaces.add("pulsar/use/shared-ns.*"); - policyData.primary = new ArrayList(); - policyData.primary.add(pulsarServices[0].getAdvertisedAddress()); - policyData.secondary = new ArrayList(); - for (int i = 1; i < BROKER_COUNT; i++) { - policyData.secondary.add(pulsarServices[i].getAdvertisedAddress()); - } - policyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - policyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - policyData.auto_failover_policy.parameters = new HashMap(); - policyData.auto_failover_policy.parameters.put("min_limit", "1"); - policyData.auto_failover_policy.parameters.put("usage_threshold", "100"); + policyData = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("pulsar/use/shared-ns.*")) + .primary(Collections.singletonList(pulsarServices[0].getWebServiceAddress())) + .secondary(allExceptFirstBroker) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build()) + .build(); policies.setPolicy("otherBrokerPolicy", policyData); String path = AdminResource.path("clusters", "use", "namespaceIsolationPolicies"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java index 575a418d6aadd..922544339d0bb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java @@ -68,6 +68,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.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -493,7 +494,7 @@ public void testNamespaceIsolationPoliciesForPrimaryAndSecondaryBrokers() throws final String broker1Address = pulsar1.getAdvertisedAddress() + "0"; final String broker2Address = pulsar2.getAdvertisedAddress() + "1"; final String sharedBroker = "broker3"; - admin1.clusters().createCluster(cluster, new ClusterDataImpl("http://" + pulsar1.getAdvertisedAddress())); + admin1.clusters().createCluster(cluster, ClusterData.builder().serviceUrl("http://" + pulsar1.getAdvertisedAddress()).build()); admin1.tenants().createTenant(tenant, new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet(cluster))); admin1.namespaces().createNamespace(tenant + "/" + cluster + "/" + namespace); 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 5e19d4fa0b434..10ddc236088ab 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 @@ -36,6 +36,7 @@ import java.lang.reflect.Field; import java.net.URL; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -67,8 +68,10 @@ import org.apache.pulsar.client.admin.PulsarAdmin; 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.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.ResourceQuota; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; @@ -172,20 +175,21 @@ void shutdown() throws Exception { } private void createNamespacePolicies(PulsarService pulsar) throws Exception { + Map parameters = new HashMap<>(); + parameters.put("min_limit", "1"); + parameters.put("usage_threshold", "100"); + NamespaceIsolationPolicies policies = new NamespaceIsolationPolicies(); // set up policy that use this broker as primary - NamespaceIsolationDataImpl policyData = new NamespaceIsolationDataImpl(); - policyData.namespaces = new ArrayList<>(); - policyData.namespaces.add("pulsar/use/primary-ns.*"); - policyData.primary = new ArrayList<>(); - policyData.primary.add(pulsar1.getAdvertisedAddress() + "*"); - policyData.secondary = new ArrayList<>(); - policyData.secondary.add("prod2-broker([78]).messaging.usw.example.co.*"); - policyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - policyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - policyData.auto_failover_policy.parameters = new HashMap<>(); - policyData.auto_failover_policy.parameters.put("min_limit", "1"); - policyData.auto_failover_policy.parameters.put("usage_threshold", "100"); + NamespaceIsolationData policyData = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("pulsar/use/primary-ns.*")) + .primary(Collections.singletonList(pulsar1.getAdvertisedAddress() + "*")) + .secondary(Collections.singletonList("prod2-broker([78]).messaging.usw.example.co.*")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build()) + .build(); policies.setPolicy("primaryBrokerPolicy", policyData); String path = AdminResource.path("clusters", "use", "namespaceIsolationPolicies"); 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 a5642528700c1..0f2977e71f80f 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 @@ -51,6 +51,7 @@ import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.zookeeper.ZooKeeperChildrenCache; @@ -93,9 +94,9 @@ public void setUp() throws Exception { clusters.add("use"); clusters.add("usc"); clusters.add("usw"); - ClusterDataImpl useData = new ClusterDataImpl("http://broker.messaging.use.example.com:8080"); - ClusterDataImpl uscData = new ClusterDataImpl("http://broker.messaging.usc.example.com:8080"); - ClusterDataImpl uswData = new ClusterDataImpl("http://broker.messaging.usw.example.com:8080"); + ClusterData useData = ClusterData.builder().serviceUrl("http://broker.messaging.use.example.com:8080").build(); + ClusterData uscData = ClusterData.builder().serviceUrl("http://broker.messaging.usc.example.com:8080").build(); + ClusterData uswData = ClusterData.builder().serviceUrl("http://broker.messaging.usw.example.com:8080").build(); doReturn(config).when(pulsar).getConfiguration(); doReturn(mockConfigCache).when(pulsar).getConfigurationCache(); doReturn(clustersListCache).when(mockConfigCache).clustersListCache(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java index d0dd3d48c3565..23e665b414908 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java @@ -56,8 +56,8 @@ public void testCreateNamespaceWithDefaultBundles() throws Exception { admin.namespaces().createNamespace(namespaceName); Policies policies = admin.namespaces().getPolicies(namespaceName); - assertEquals(policies.bundles.numBundles, 16); - assertEquals(policies.bundles.boundaries.size(), 17); + assertEquals(policies.bundles.getNumBundles(), 16); + assertEquals(policies.bundles.getBoundaries().size(), 17); } @Test @@ -73,8 +73,9 @@ public void testSplitBundleUpdatesLocalPoliciesWithoutOverwriting() throws Excep Producer producer = producerBuilder.create(); String bundle = admin.lookups().getBundleRange(topicName); - BookieAffinityGroupData bookieAffinityGroup = new BookieAffinityGroupData(); - bookieAffinityGroup.bookkeeperAffinityGroupPrimary = "test"; + BookieAffinityGroupData bookieAffinityGroup = BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary("test") + .build(); admin.namespaces().setBookieAffinityGroup(namespaceName, bookieAffinityGroup); admin.namespaces().splitNamespaceBundle(namespaceName, bundle, false, null); assertNotNull(admin.namespaces().getBookieAffinityGroup(namespaceName)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java index 7c85c3a1d8474..9c2ac0ad4d3fd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Sets; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -40,7 +41,7 @@ protected void setup() throws Exception { internalSetup(); String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NAMESPACE); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupServiceTest.java index 2f9a00c1e39ae..6024e57ea36a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupServiceTest.java @@ -29,6 +29,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -245,6 +246,6 @@ public void testResourceGroupOps() throws PulsarAdminException, InterruptedExcep private static final int PUBLISH_INTERVAL_SECS = 500; private void prepareData() throws PulsarAdminException { this.conf.setResourceUsageTransportPublishIntervalInSecs(PUBLISH_INTERVAL_SECS); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java index de5aee659d5cb..e2afb14fb2496 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java @@ -32,9 +32,11 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -190,11 +192,11 @@ private void verfyStats(String topicString, String rgName, boolean checkProduce, boolean checkConsume) throws InterruptedException, PulsarAdminException { BrokerService bs = pulsar.getBrokerService(); - Map topicStatsMap = bs.getTopicStats(); - for (Map.Entry entry : topicStatsMap.entrySet()) { + Map topicStatsMap = bs.getTopicStats(); + for (Map.Entry entry : topicStatsMap.entrySet()) { String mapTopicName = entry.getKey(); if (mapTopicName.equals(topicString)) { - TopicStats stats = entry.getValue(); + TopicStatsImpl stats = entry.getValue(); if (checkProduce) { Assert.assertTrue(stats.bytesInCounter >= sentNumBytes); Assert.assertTrue(stats.msgInCounter == sentNumMsgs); @@ -249,7 +251,7 @@ private void prepareData() throws PulsarAdminException { this.conf.setAllowAutoTopicCreation(true); final String clusterName = "test"; - admin.clusters().createCluster(clusterName, new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster(clusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant(TenantName, new TenantInfoImpl(Sets.newHashSet("fakeAdminRole"), Sets.newHashSet(clusterName))); admin.namespaces().createNamespace(TenantAndNsName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java index 1f9f735c9c078..a6fa17ff57db6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java @@ -23,6 +23,7 @@ import org.apache.pulsar.broker.service.resource.usage.ResourceUsage; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -115,6 +116,6 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { private void prepareData() throws PulsarAdminException { this.conf.setResourceUsageTransportClassName("org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager"); this.conf.setResourceUsageTransportPublishIntervalInSecs(PUBLISH_INTERVAL_SECS); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index f960e01c55499..552ba5a7a0b8a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -46,6 +46,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -99,7 +100,7 @@ void setup() throws Exception { adminUrl = new URL("http://127.0.0.1" + ":" + pulsar.getListenPortHTTP().get()); admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); - admin.clusters().createCluster("usc", new ClusterDataImpl(adminUrl.toString())); + admin.clusters().createCluster("usc", ClusterData.builder().serviceUrl(adminUrl.toString()).build()); admin.tenants().createTenant("prop", new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("usc"))); admin.namespaces().createNamespace("prop/ns-quota"); @@ -147,7 +148,11 @@ public void testBacklogQuotaWithReader() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic1"; final int numMsgs = 20; @@ -169,11 +174,11 @@ public void testBacklogQuotaWithReader() throws Exception { TopicStats stats = admin.topics().getStats(topic1); // overall backlogSize should be zero because we only have readers - assertEquals(stats.backlogSize, 0, "backlog size is [" + stats.backlogSize + "]"); + assertEquals(stats.getBacklogSize(), 0, "backlog size is [" + stats.getBacklogSize() + "]"); // non-durable mes should still - assertEquals(stats.subscriptions.size(), 1); - long nonDurableSubscriptionBacklog = stats.subscriptions.values().iterator().next().msgBacklog; + assertEquals(stats.getSubscriptions().size(), 1); + long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); assertEquals(nonDurableSubscriptionBacklog, MAX_ENTRIES_PER_LEDGER, "non-durable subscription backlog is [" + nonDurableSubscriptionBacklog + "]"); ; @@ -220,7 +225,11 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; @@ -236,10 +245,10 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); // overall backlogSize should be zero because we only have readers - assertEquals(stats.backlogSize, 0, "backlog size is [" + stats.backlogSize + "]"); + assertEquals(stats.getBacklogSize(), 0, "backlog size is [" + stats.getBacklogSize() + "]"); // non-durable mes should still - assertEquals(stats.subscriptions.size(), 1); - long nonDurableSubscriptionBacklog = stats.subscriptions.values().iterator().next().msgBacklog; + assertEquals(stats.getSubscriptions().size(), 1); + long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); assertEquals(nonDurableSubscriptionBacklog, MAX_ENTRIES_PER_LEDGER, "non-durable subscription backlog is [" + nonDurableSubscriptionBacklog + "]"); ; try { @@ -286,7 +295,11 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); final int numMsgs = 9; @@ -302,10 +315,10 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); // overall backlogSize should be zero because we only have readers - assertEquals(stats.backlogSize, 0, "backlog size is [" + stats.backlogSize + "]"); + assertEquals(stats.getBacklogSize(), 0, "backlog size is [" + stats.getBacklogSize() + "]"); // non-durable mes should still - assertEquals(stats.subscriptions.size(), 1); - long nonDurableSubscriptionBacklog = stats.subscriptions.values().iterator().next().msgBacklog; + assertEquals(stats.getSubscriptions().size(), 1); + long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); // non-durable subscription won't trigger the check for time based backlog quota // and cause back pressure action to be token. Since broker don't keep track consuming position for reader. assertEquals(nonDurableSubscriptionBacklog, numMsgs, @@ -349,7 +362,10 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) .build(); @@ -373,7 +389,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); - assertTrue(stats.backlogSize < 10 * 1024, "Storage size is [" + stats.storageSize + "]"); + assertTrue(stats.getBacklogSize() < 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); } @Test @@ -381,7 +397,11 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(20 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(20 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); config.setPreciseTimeBasedBacklogQuotaCheck(true); PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) .build(); @@ -402,16 +422,16 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { } TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 9); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 9); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 9); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 9); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA * 2) * 1000); rolloverStats(); stats = admin.topics().getStats(topic1); // All messages for both subscription should be cleaned up from backlog by backlog monitor task. - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 0); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 0); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 0); client.close(); } @@ -420,7 +440,11 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(20 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(20 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) .build(); @@ -440,8 +464,8 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { } TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 14); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 14); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 14); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 14); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA * 2) * 1000); rolloverStats(); @@ -449,8 +473,8 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { stats = admin.topics().getStats(topic1); // Messages on first 2 ledgers should be expired, backlog is number of // message in current ledger which should be 4. - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 4); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 4); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 4); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 4); client.close(); } @@ -459,7 +483,11 @@ public void testConsumerBacklogEvictionWithAckSizeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).build(); final String topic1 = "persistent://prop/ns-quota/topic11"; @@ -482,7 +510,7 @@ public void testConsumerBacklogEvictionWithAckSizeQuota() throws Exception { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); - assertTrue(stats.backlogSize <= 10 * 1024, "Storage size is [" + stats.storageSize + "]"); + assertTrue(stats.getBacklogSize() <= 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); } @Test @@ -490,7 +518,11 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); config.setPreciseTimeBasedBacklogQuotaCheck(true); PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).build(); @@ -511,8 +543,8 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio } TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 9); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 9); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 9); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 9); consumer1.redeliverUnacknowledgedMessages(); for (int i = 0; i < numMsgs; i++) { @@ -524,15 +556,15 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio rolloverStats(); stats = admin.topics().getStats(topic1); // sub1 has empty backlog as it acked all messages - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 0); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 9); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 9); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA * 2) * 1000); rolloverStats(); stats = admin.topics().getStats(topic1); // sub2 has empty backlog because it's backlog get cleaned up by backlog quota monitor task - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 0); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 0); client.close(); } @@ -550,8 +582,11 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(20 * 1024, 2 * TIME_TO_CHECK_BACKLOG_QUOTA, - BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(20 * 1024) + .limitTime(2 * TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).build(); @@ -575,8 +610,8 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { { TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 14); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 14); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 14); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 14); } for (int i = 0; i < numMsgs; i++) { @@ -595,8 +630,8 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); // sub1 has empty backlog as it acked all messages - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 0); - assertEquals(stats.subscriptions.get(subName2).msgBacklog, 14); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); + assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 14); }); Awaitility.await() @@ -605,7 +640,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { .untilAsserted(() -> { // Messages on first 2 ledgers should be expired, backlog is number of // message in current ledger which should be 4. - long msgBacklog = admin.topics().getStats(topic1).subscriptions.get(subName2).msgBacklog; + long msgBacklog = admin.topics().getStats(topic1).getSubscriptions().get(subName2).getMsgBacklog(); // TODO: for some reason the backlog size is sometimes off by one // Internally there's a method `long getNumberOfEntriesInBacklog(boolean getPreciseBacklog)` // on org.apache.pulsar.broker.service.Subscription interface @@ -619,7 +654,10 @@ public void testConcurrentAckAndEviction() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); final String topic1 = "persistent://prop/ns-quota/topic12"; final String subName1 = "c12"; @@ -683,7 +721,7 @@ public void run() { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); - assertTrue(stats.backlogSize <= 10 * 1024, "Storage size is [" + stats.storageSize + "]"); + assertTrue(stats.getBacklogSize() <= 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); } @Test @@ -691,7 +729,10 @@ public void testNoEviction() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); final String topic1 = "persistent://prop/ns-quota/topic13"; final String subName1 = "c13"; @@ -756,7 +797,10 @@ public void testEvictionMulti() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/ns-quota", - new BacklogQuota(15 * 1024, BacklogQuota.RetentionPolicy.consumer_backlog_eviction)); + BacklogQuota.builder() + .limitSize(15 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) + .build()); final String topic1 = "persistent://prop/ns-quota/topic14"; final String subName1 = "c14"; @@ -855,7 +899,7 @@ public void run() { rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); - assertTrue(stats.backlogSize <= 15 * 1024, "Storage size is [" + stats.storageSize + "]"); + assertTrue(stats.getBacklogSize() <= 15 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); } @Test @@ -863,7 +907,10 @@ public void testAheadProducerOnHold() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/quotahold", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.producer_request_hold)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_request_hold) + .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) .statsInterval(0, TimeUnit.SECONDS).build(); @@ -893,8 +940,8 @@ public void testAheadProducerOnHold() throws Exception { Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.publishers.size(), 0, - "Number of producers on topic " + topic1 + " are [" + stats.publishers.size() + "]"); + assertEquals(stats.getPublishers().size(), 0, + "Number of producers on topic " + topic1 + " are [" + stats.getPublishers().size() + "]"); } @Test @@ -902,7 +949,10 @@ public void testAheadProducerOnHoldTimeout() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/quotahold", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.producer_request_hold)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_request_hold) + .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) .statsInterval(0, TimeUnit.SECONDS).build(); @@ -937,7 +987,10 @@ public void testProducerException() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/quotahold", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) .statsInterval(0, TimeUnit.SECONDS).build(); @@ -974,7 +1027,10 @@ public void testProducerExceptionAndThenUnblockSizeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/quotahold", - new BacklogQuota(10 * 1024, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) .statsInterval(0, TimeUnit.SECONDS).build(); @@ -1007,7 +1063,7 @@ public void testProducerExceptionAndThenUnblockSizeQuota() throws Exception { // now remove backlog and ensure that producer is unblocked; TopicStats stats = admin.topics().getStats(topic1); - int backlog = (int) stats.subscriptions.get(subName1).msgBacklog; + int backlog = (int) stats.getSubscriptions().get(subName1).getMsgBacklog(); for (int i = 0; i < backlog; i++) { Message msg = consumer.receive(); @@ -1033,7 +1089,11 @@ public void testProducerExceptionAndThenUnblockTimeQuotaPrecise() throws Excepti assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/quotahold", - new BacklogQuota(10 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); config.setPreciseTimeBasedBacklogQuotaCheck(true); final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) .statsInterval(0, TimeUnit.SECONDS).build(); @@ -1066,7 +1126,7 @@ public void testProducerExceptionAndThenUnblockTimeQuotaPrecise() throws Excepti // now remove backlog and ensure that producer is unblocked; TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, numMsgs); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), numMsgs); for (int i = 0; i < numMsgs; i++) { consumer.acknowledge(consumer.receive()); @@ -1075,7 +1135,7 @@ public void testProducerExceptionAndThenUnblockTimeQuotaPrecise() throws Excepti Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA * 2) * 1000); rolloverStats(); stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 0); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); // publish should work now Exception sendException = null; gotException = false; @@ -1096,7 +1156,11 @@ public void testProducerExceptionAndThenUnblockTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), Maps.newHashMap()); admin.namespaces().setBacklogQuota("prop/quotahold", - new BacklogQuota(15 * 1024, TIME_TO_CHECK_BACKLOG_QUOTA, BacklogQuota.RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(15 * 1024) + .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build()); final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) .statsInterval(0, TimeUnit.SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; @@ -1128,7 +1192,7 @@ public void testProducerExceptionAndThenUnblockTimeQuota() throws Exception { // now remove backlog and ensure that producer is unblocked; TopicStats stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, numMsgs); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), numMsgs); for (int i = 0; i < numMsgs; i++) { consumer.acknowledge(consumer.receive()); @@ -1137,7 +1201,7 @@ public void testProducerExceptionAndThenUnblockTimeQuota() throws Exception { Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA * 2) * 1000); rolloverStats(); stats = admin.topics().getStats(topic1); - assertEquals(stats.subscriptions.get(subName1).msgBacklog, 0); + assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); // publish should work now Exception sendException = null; gotException = false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java index fbc4a9986fd55..ff9ee31fd6913 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.tests.TestRetrySupport; @@ -98,7 +99,7 @@ protected void setup() throws Exception { admin = PulsarAdmin.builder().serviceHttpUrl(pulsar.getWebServiceAddress()).build(); - admin.clusters().createCluster("usc", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("usc", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("prop", new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("usc"))); } catch (Throwable t) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index 1a7df976d93d7..445e3c377b02a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -62,6 +62,7 @@ import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.BookieInfo; import org.apache.pulsar.common.policies.data.BookiesRackConfiguration; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -167,7 +168,7 @@ public void testBookieIsolation() throws Exception { PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarService.getWebServiceAddress()).build(); - ClusterDataImpl clusterData = new ClusterDataImpl(pulsarService.getWebServiceAddress()); + ClusterData clusterData = ClusterData.builder().serviceUrl(pulsarService.getWebServiceAddress()).build(); admin.clusters().createCluster(cluster, clusterData); TenantInfoImpl tenantInfo = new TenantInfoImpl(null, Sets.newHashSet(cluster)); admin.tenants().createTenant(tenant1, tenantInfo); @@ -176,18 +177,30 @@ public void testBookieIsolation() throws Exception { admin.namespaces().createNamespace(ns3); admin.namespaces().createNamespace(ns4); admin.namespaces().setBookieAffinityGroup(ns2, - new BookieAffinityGroupData(tenantNamespaceIsolationGroups, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroups) + .build()); admin.namespaces().setBookieAffinityGroup(ns3, - new BookieAffinityGroupData(tenantNamespaceIsolationGroups, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroups) + .build()); admin.namespaces().setBookieAffinityGroup(ns4, - new BookieAffinityGroupData(tenantNamespaceIsolationGroups, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroups) + .build()); assertEquals(admin.namespaces().getBookieAffinityGroup(ns2), - new BookieAffinityGroupData(tenantNamespaceIsolationGroups, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroups) + .build()); assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), - new BookieAffinityGroupData(tenantNamespaceIsolationGroups, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroups) + .build()); assertEquals(admin.namespaces().getBookieAffinityGroup(ns4), - new BookieAffinityGroupData(tenantNamespaceIsolationGroups, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroups) + .build()); try { admin.namespaces().getBookieAffinityGroup(ns1); @@ -310,7 +323,7 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarService.getWebServiceAddress()).build(); - ClusterDataImpl clusterData = new ClusterDataImpl(pulsarService.getWebServiceAddress()); + ClusterData clusterData = ClusterData.builder().serviceUrl(pulsarService.getWebServiceAddress()).build(); admin.clusters().createCluster(cluster, clusterData); TenantInfoImpl tenantInfo = new TenantInfoImpl(null, Sets.newHashSet(cluster)); admin.tenants().createTenant(tenant1, tenantInfo); @@ -318,19 +331,35 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { admin.namespaces().createNamespace(ns2); admin.namespaces().createNamespace(ns3); admin.namespaces().createNamespace(ns4); - admin.namespaces().setBookieAffinityGroup(ns2, new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); - admin.namespaces().setBookieAffinityGroup(ns3, new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); + admin.namespaces().setBookieAffinityGroup(ns2, + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); + admin.namespaces().setBookieAffinityGroup(ns3, + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); admin.namespaces().setBookieAffinityGroup(ns4, - new BookieAffinityGroupData(tenantNamespaceIsolationGroupsPrimary, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .build()); - assertEquals(admin.namespaces().getBookieAffinityGroup(ns2), new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); - assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); + assertEquals(admin.namespaces().getBookieAffinityGroup(ns2), + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); + assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); assertEquals(admin.namespaces().getBookieAffinityGroup(ns4), - new BookieAffinityGroupData(tenantNamespaceIsolationGroupsPrimary, null)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .build()); try { admin.namespaces().getBookieAffinityGroup(ns1); @@ -432,7 +461,7 @@ public void testDeleteIsolationGroup() throws Exception { PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarService.getWebServiceAddress()).build(); - ClusterDataImpl clusterData = new ClusterDataImpl(pulsarService.getWebServiceAddress()); + ClusterData clusterData = ClusterData.builder().serviceUrl(pulsarService.getWebServiceAddress()).build(); admin.clusters().createCluster(cluster, clusterData); TenantInfoImpl tenantInfo = new TenantInfoImpl(null, Sets.newHashSet(cluster)); admin.tenants().createTenant(tenant1, tenantInfo); @@ -440,16 +469,28 @@ public void testDeleteIsolationGroup() throws Exception { admin.namespaces().createNamespace(ns3); // (1) set affinity-group - admin.namespaces().setBookieAffinityGroup(ns2, new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); - admin.namespaces().setBookieAffinityGroup(ns3, new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); + admin.namespaces().setBookieAffinityGroup(ns2, + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); + admin.namespaces().setBookieAffinityGroup(ns3, + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); // (2) get affinity-group - assertEquals(admin.namespaces().getBookieAffinityGroup(ns2), new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); - assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); + assertEquals(admin.namespaces().getBookieAffinityGroup(ns2), + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); + assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); // (3) delete affinity-group admin.namespaces().deleteBookieAffinityGroup(ns2); @@ -461,8 +502,11 @@ public void testDeleteIsolationGroup() throws Exception { // Ok } - assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), new BookieAffinityGroupData( - tenantNamespaceIsolationGroupsPrimary, tenantNamespaceIsolationGroupsSecondary)); + assertEquals(admin.namespaces().getBookieAffinityGroup(ns3), + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(tenantNamespaceIsolationGroupsPrimary) + .bookkeeperAffinityGroupSecondary(tenantNamespaceIsolationGroupsSecondary) + .build()); } @@ -517,7 +561,7 @@ private void setDefaultIsolationGroup(String brokerBookkeeperClientIsolationGrou Map bookieInfoMap = Maps.newHashMap(); for (BookieId bkSocket : bookieAddresses) { - BookieInfo info = new BookieInfo("use", bkSocket.toString()); + BookieInfo info = BookieInfo.builder().rack("use").hostname(bkSocket.toString()).build(); bookieInfoMap.put(bkSocket.toString(), info); } bookies.put(brokerBookkeeperClientIsolationGroups, bookieInfoMap); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java index 0050d99389dbd..c635968907ab5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java @@ -99,7 +99,9 @@ public void testAutoSubscriptionCreationNamespaceAllowOverridesBroker() throws E pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); pulsar.getAdminClient().namespaces().setAutoSubscriptionCreation(topicName.getNamespace(), - new AutoSubscriptionCreationOverride(true)); + AutoSubscriptionCreationOverride.builder() + .allowAutoSubscriptionCreation(true) + .build()); // Subscribe operation should be successful pulsarClient.newConsumer().topic(topicName.toString()).subscriptionName(subscriptionName).subscribe(); @@ -116,7 +118,9 @@ public void testAutoSubscriptionCreationNamespaceDisallowOverridesBroker() throw pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); pulsar.getAdminClient().namespaces().setAutoSubscriptionCreation(topicName.getNamespace(), - new AutoSubscriptionCreationOverride(false)); + AutoSubscriptionCreationOverride.builder() + .allowAutoSubscriptionCreation(false) + .build()); try { pulsarClient.newConsumer().topic(topicName.toString()).subscriptionName(subscriptionName).subscribe(); 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 3275f5e4a2383..f7fffda057fcc 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 @@ -168,7 +168,10 @@ public void testAutoCreationNamespaceAllowOverridesBroker() throws Exception { final TopicName topicName = TopicName.get(topicString); pulsar.getConfiguration().setAllowAutoTopicCreation(false); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.NON_PARTITIONED.toString()) + .build()); pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString)); @@ -182,7 +185,9 @@ public void testAutoCreationNamespaceDisallowOverridesBroker() throws Exception final TopicName topicName = TopicName.get(topicString); pulsar.getConfiguration().setAllowAutoTopicCreation(true); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(false, null, null)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(false) + .build()); try { pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); @@ -200,7 +205,11 @@ public void testAutoCreationNamespaceOverrideAllowsPartitionedTopics() throws Ex pulsar.getConfiguration().setAllowAutoTopicCreation(false); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 4)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .defaultNumPartitions(4) + .build()); final String subscriptionName = "test-topic-sub-6"; pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); @@ -219,7 +228,11 @@ public void testAutoCreationNamespaceOverridesTopicTypePartitioned() throws Exce pulsar.getConfiguration().setAllowAutoTopicCreation(true); pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned"); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 3)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .defaultNumPartitions(3) + .build()); final String subscriptionName = "test-topic-sub-7"; pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); @@ -239,7 +252,10 @@ public void testAutoCreationNamespaceOverridesTopicTypeNonPartitioned() throws E pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned"); pulsar.getConfiguration().setDefaultNumPartitions(2); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.NON_PARTITIONED.toString()) + .build()); final String subscriptionName = "test-topic-sub-8"; pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); @@ -257,7 +273,11 @@ public void testAutoCreationNamespaceOverridesDefaultNumPartitions() throws Exce pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned"); pulsar.getConfiguration().setDefaultNumPartitions(2); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 4)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .defaultNumPartitions(4) + .build()); final String subscriptionName = "test-topic-sub-9"; @@ -275,7 +295,10 @@ public void testAutoCreationNamespaceAllowOverridesBrokerOnProduce() throws Exce final TopicName topicName = TopicName.get(topicString); pulsar.getConfiguration().setAllowAutoTopicCreation(false); pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(), - new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.NON_PARTITIONED.toString()) + .build()); pulsarClient.newProducer().topic(topicString).create(); assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString)); 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 380786abd726a..a0e3a9ef0d39b 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 @@ -170,15 +170,15 @@ public void testBrokerServicePersistentTopicStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); // subscription stats - assertEquals(stats.subscriptions.keySet().size(), 1); - assertEquals(subStats.msgBacklog, 0); - assertEquals(subStats.consumers.size(), 1); + assertEquals(stats.getSubscriptions().keySet().size(), 1); + assertEquals(subStats.getMsgBacklog(), 0); + assertEquals(subStats.getConsumers().size(), 1); // storage stats - assertEquals(stats.offloadedStorageSize, 0); + assertEquals(stats.getOffloadedStorageSize(), 0); Producer producer = pulsarClient.newProducer().topic(topicName).create(); Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @@ -191,33 +191,33 @@ public void testBrokerServicePersistentTopicStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); // publisher stats - assertEquals(subStats.msgBacklog, 10); - assertEquals(stats.publishers.size(), 1); - assertTrue(stats.publishers.get(0).msgRateIn > 0.0); - assertTrue(stats.publishers.get(0).msgThroughputIn > 0.0); - assertTrue(stats.publishers.get(0).averageMsgSize > 0.0); - assertNotNull(stats.publishers.get(0).getClientVersion()); + assertEquals(subStats.getMsgBacklog(), 10); + assertEquals(stats.getPublishers().size(), 1); + assertTrue(stats.getPublishers().get(0).getMsgRateIn() > 0.0); + assertTrue(stats.getPublishers().get(0).getMsgThroughputIn() > 0.0); + assertTrue(stats.getPublishers().get(0).getAverageMsgSize() > 0.0); + assertNotNull(stats.getPublishers().get(0).getClientVersion()); // aggregated publish stats - assertEquals(stats.msgRateIn, stats.publishers.get(0).msgRateIn); - assertEquals(stats.msgThroughputIn, stats.publishers.get(0).msgThroughputIn); - double diff = stats.averageMsgSize - stats.publishers.get(0).averageMsgSize; + assertEquals(stats.getMsgRateIn(), stats.getPublishers().get(0).getMsgRateIn()); + assertEquals(stats.getMsgThroughputIn(), stats.getPublishers().get(0).getMsgThroughputIn()); + double diff = stats.getAverageMsgSize() - stats.getPublishers().get(0).getAverageMsgSize(); assertTrue(Math.abs(diff) < 0.000001); // consumer stats - assertTrue(subStats.consumers.get(0).msgRateOut > 0.0); - assertTrue(subStats.consumers.get(0).msgThroughputOut > 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgRateOut() > 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgThroughputOut() > 0.0); // aggregated consumer stats - assertEquals(subStats.msgRateOut, subStats.consumers.get(0).msgRateOut); - assertEquals(subStats.msgThroughputOut, subStats.consumers.get(0).msgThroughputOut); - assertEquals(stats.msgRateOut, subStats.consumers.get(0).msgRateOut); - assertEquals(stats.msgThroughputOut, subStats.consumers.get(0).msgThroughputOut); - assertNotNull(subStats.consumers.get(0).getClientVersion()); - assertEquals(stats.offloadedStorageSize, 0); + assertEquals(subStats.getMsgRateOut(), subStats.getConsumers().get(0).getMsgRateOut()); + assertEquals(subStats.getMsgThroughputOut(), subStats.getConsumers().get(0).getMsgThroughputOut()); + assertEquals(stats.getMsgRateOut(), subStats.getConsumers().get(0).getMsgRateOut()); + assertEquals(stats.getMsgThroughputOut(), subStats.getConsumers().get(0).getMsgThroughputOut()); + assertNotNull(subStats.getConsumers().get(0).getClientVersion()); + assertEquals(stats.getOffloadedStorageSize(), 0); Message msg; for (int i = 0; i < 10; i++) { @@ -229,10 +229,10 @@ public void testBrokerServicePersistentTopicStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); - assertEquals(stats.offloadedStorageSize, 0); + subStats = stats.getSubscriptions().values().iterator().next(); + assertEquals(stats.getOffloadedStorageSize(), 0); - assertEquals(subStats.msgBacklog, 0); + assertEquals(subStats.getMsgBacklog(), 0); } @Test @@ -271,12 +271,12 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); // subscription stats - assertEquals(stats.subscriptions.keySet().size(), 1); - assertEquals(subStats.msgBacklog, 0); - assertEquals(subStats.consumers.size(), 1); + assertEquals(stats.getSubscriptions().keySet().size(), 1); + assertEquals(subStats.getMsgBacklog(), 0); + assertEquals(subStats.getConsumers().size(), 1); Producer producer = pulsarClient.newProducer().topic(topicName).create(); Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @@ -289,44 +289,44 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); // publisher stats - assertEquals(subStats.msgBacklog, 10); - assertEquals(stats.publishers.size(), 1); - assertTrue(stats.publishers.get(0).msgRateIn > 0.0); - assertTrue(stats.publishers.get(0).msgThroughputIn > 0.0); - assertTrue(stats.publishers.get(0).averageMsgSize > 0.0); + assertEquals(subStats.getMsgBacklog(), 10); + assertEquals(stats.getPublishers().size(), 1); + assertTrue(stats.getPublishers().get(0).getMsgRateIn() > 0.0); + assertTrue(stats.getPublishers().get(0).getMsgThroughputIn() > 0.0); + assertTrue(stats.getPublishers().get(0).getAverageMsgSize() > 0.0); // aggregated publish stats - assertEquals(stats.msgRateIn, stats.publishers.get(0).msgRateIn); - assertEquals(stats.msgThroughputIn, stats.publishers.get(0).msgThroughputIn); - double diff = stats.averageMsgSize - stats.publishers.get(0).averageMsgSize; + assertEquals(stats.getMsgRateIn(), stats.getPublishers().get(0).getMsgRateIn()); + assertEquals(stats.getMsgThroughputIn(), stats.getPublishers().get(0).getMsgThroughputIn()); + double diff = stats.getAverageMsgSize() - stats.getPublishers().get(0).getAverageMsgSize(); assertTrue(Math.abs(diff) < 0.000001); // consumer stats - assertTrue(subStats.consumers.get(0).msgRateOut > 0.0); - assertTrue(subStats.consumers.get(0).msgThroughputOut > 0.0); - assertEquals(subStats.msgRateRedeliver, 0.0); - assertEquals(subStats.consumers.get(0).unackedMessages, 10); + assertTrue(subStats.getConsumers().get(0).getMsgRateOut() > 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgThroughputOut() > 0.0); + assertEquals(subStats.getMsgRateRedeliver(), 0.0); + assertEquals(subStats.getConsumers().get(0).getUnackedMessages(), 10); // aggregated consumer stats - assertEquals(subStats.msgRateOut, subStats.consumers.get(0).msgRateOut); - assertEquals(subStats.msgThroughputOut, subStats.consumers.get(0).msgThroughputOut); - assertEquals(subStats.msgRateRedeliver, subStats.consumers.get(0).msgRateRedeliver); - assertEquals(stats.msgRateOut, subStats.consumers.get(0).msgRateOut); - assertEquals(stats.msgThroughputOut, subStats.consumers.get(0).msgThroughputOut); - assertEquals(subStats.msgRateRedeliver, subStats.consumers.get(0).msgRateRedeliver); - assertEquals(subStats.unackedMessages, subStats.consumers.get(0).unackedMessages); + assertEquals(subStats.getMsgRateOut(), subStats.getConsumers().get(0).getMsgRateOut()); + assertEquals(subStats.getMsgThroughputOut(), subStats.getConsumers().get(0).getMsgThroughputOut()); + assertEquals(subStats.getMsgRateRedeliver(), subStats.getConsumers().get(0).getMsgRateRedeliver()); + assertEquals(stats.getMsgRateOut(), subStats.getConsumers().get(0).getMsgRateOut()); + assertEquals(stats.getMsgThroughputOut(), subStats.getConsumers().get(0).getMsgThroughputOut()); + assertEquals(subStats.getMsgRateRedeliver(), subStats.getConsumers().get(0).getMsgRateRedeliver()); + assertEquals(subStats.getUnackedMessages(), subStats.getConsumers().get(0).getUnackedMessages()); consumer.redeliverUnacknowledgedMessages(); Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); - assertTrue(subStats.msgRateRedeliver > 0.0); - assertEquals(subStats.msgRateRedeliver, subStats.consumers.get(0).msgRateRedeliver); + subStats = stats.getSubscriptions().values().iterator().next(); + assertTrue(subStats.getMsgRateRedeliver() > 0.0); + assertEquals(subStats.getMsgRateRedeliver(), subStats.getConsumers().get(0).getMsgRateRedeliver()); Message msg; for (int i = 0; i < 10; i++) { @@ -338,9 +338,9 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); - assertEquals(subStats.msgBacklog, 0); + assertEquals(subStats.getMsgBacklog(), 0); } @Test @@ -944,13 +944,13 @@ public void testCreateNamespacePolicy() throws Exception { final String namespace = "prop/testPolicy"; final int totalBundle = 3; System.err.println("----------------"); - admin.namespaces().createNamespace(namespace, new BundlesData(totalBundle)); + admin.namespaces().createNamespace(namespace, BundlesData.builder().numBundles(totalBundle).build()); String globalPath = joinPath(LOCAL_POLICIES_ROOT, namespace); pulsar.getLocalZkCacheService().policiesCache().clear(); Optional policy = pulsar.getLocalZkCacheService().policiesCache().get(globalPath); assertTrue(policy.isPresent()); - assertEquals(policy.get().bundles.numBundles, totalBundle); + assertEquals(policy.get().bundles.getNumBundles(), totalBundle); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerTestBase.java index b00ac20d80f52..a131b66bfca8a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerTestBase.java @@ -20,6 +20,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.slf4j.Logger; @@ -43,7 +44,7 @@ public void baseSetup(ServiceConfiguration serviceConfiguration) throws Exceptio } private void baseSetupCommon() throws Exception { - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("prop", new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("prop/ns-abc"); 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 46844ab1fe76b..c901731c79936 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 @@ -93,14 +93,14 @@ public void testProxyProtocol() throws PulsarClientException, ExecutionException 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())); TopicStats topicStats = admin.topics().getStats(topicName); - Assert.assertEquals(topicStats.subscriptions.size(), 1); - SubscriptionStats subscriptionStats = topicStats.subscriptions.get(subName); - Assert.assertEquals(subscriptionStats.consumers.size(), 1); - Assert.assertEquals(subscriptionStats.consumers.get(0).getAddress(), "198.51.100.22:35646"); + 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"); pulsarClient.newProducer().topic(topicName).create(); topicStats = admin.topics().getStats(topicName); - Assert.assertEquals(topicStats.publishers.size(), 1); - Assert.assertEquals(topicStats.publishers.get(0).getAddress(), "198.51.100.22:35646"); + Assert.assertEquals(topicStats.getPublishers().size(), 1); + Assert.assertEquals(topicStats.getPublishers().get(0).getAddress(), "198.51.100.22:35646"); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java index 4b80bcb1a43f3..ce8a43feb640f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java @@ -32,6 +32,7 @@ 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.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; @@ -76,7 +77,7 @@ void setup() { String url = "http://127.0.0.1:" + pulsar.getListenPortHTTP().get(); admin = PulsarAdmin.builder().serviceHttpUrl(url).build(); - admin.clusters().createCluster("max_message_test", new ClusterDataImpl(url)); + admin.clusters().createCluster("max_message_test", ClusterData.builder().serviceUrl(url).build()); admin.tenants() .createTenant("test", new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("max_message_test"))); admin.namespaces().createNamespace("test/message", Sets.newHashSet("max_message_test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PeerReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PeerReplicatorTest.java index d48f0b4cc9a93..8381ad599f10f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PeerReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PeerReplicatorTest.java @@ -138,10 +138,10 @@ public void testPeerClusterTopicLookup(String protocol) throws Exception { // get stats for topic1 using cluster-r3's admin3 TopicStats stats = admin1.topics().getStats(topic1); assertNotNull(stats); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); stats = admin3.topics().getStats(topic1); assertNotNull(stats); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); producer.close(); // set peer-clusters : r3->r2 @@ -153,10 +153,10 @@ public void testPeerClusterTopicLookup(String protocol) throws Exception { // get stats for topic1 using cluster-r3's admin3 stats = admin3.topics().getStats(topic2); assertNotNull(stats); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); stats = admin3.topics().getStats(topic2); assertNotNull(stats); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); producer.close(); @@ -223,10 +223,10 @@ public void testPeerClusterInReplicationClusterListChange() throws Exception { // get stats for topic1 using cluster-r3's admin3 TopicStats stats = admin1.topics().getStats(topic1); assertNotNull(stats); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); stats = admin3.topics().getStats(topic1); assertNotNull(stats); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); producer.close(); // change the repl cluster to peer-cluster r3 from r1 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java index 3240a3ad8ffcf..143f2544cfed1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java @@ -496,10 +496,10 @@ public void testUnackedCountWithRedeliveries() throws Exception { Awaitility.await().untilAsserted(() -> { TopicStats stats = admin.topics().getStats(topicName); // Unacked messages count should be 0 for both consumers at this point - SubscriptionStats subStats = stats.subscriptions.get(subName); - assertEquals(subStats.msgBacklog, 0); - for (ConsumerStats cs : subStats.consumers) { - assertEquals(cs.unackedMessages, 0); + SubscriptionStats subStats = stats.getSubscriptions().get(subName); + assertEquals(subStats.getMsgBacklog(), 0); + for (ConsumerStats cs : subStats.getConsumers()) { + assertEquals(cs.getUnackedMessages(), 0); } }); 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 3732337dc04a6..9de327754e3b7 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 @@ -1713,7 +1713,7 @@ public void testFailoverSubscription() throws Exception { } /** - * {@link NonPersistentReplicator.removeReplicator} doesn't remove replicator in atomic way and does in multiple step: + * NonPersistentReplicator.removeReplicator doesn't remove replicator in atomic way and does in multiple step: * 1. disconnect replicator producer *

* 2. close cursor diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/RackAwareTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/RackAwareTest.java index d7cc9de39617b..c1d5eedc883c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/RackAwareTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/RackAwareTest.java @@ -97,7 +97,10 @@ public void testPlacement() throws Exception { // Place bookie-1 in "rack-1" and the rest in "rack-2" int rackId = i == 0 ? 1 : 2; - BookieInfo bi = new BookieInfo("rack-" + rackId, "bookie-" + (i + 1)); + BookieInfo bi = BookieInfo.builder() + .rack("rack-" + rackId) + .hostname("bookie-" + (i + 1)) + .build(); log.info("setting rack for bookie at {} -- {}", bookie, bi); admin.bookies().updateBookieRackInfo(bookie, "default", bi); } 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 ec16e509e6614..93ad914d3f997 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 @@ -106,7 +106,11 @@ public void testReplicatorRatePriority() throws Exception { assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 200L); //set namespace-level policy, which should take effect - DispatchRate nsDispatchRate = new DispatchRate(50, 60L, 70); + DispatchRate nsDispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(50) + .dispatchThrottlingRateInByte(60) + .ratePeriodInSecond(60) + .build(); admin1.namespaces().setReplicatorDispatchRate(namespace, nsDispatchRate); Awaitility.await() .untilAsserted(() -> assertEquals(admin1.namespaces().getReplicatorDispatchRate(namespace), nsDispatchRate)); @@ -114,7 +118,11 @@ public void testReplicatorRatePriority() throws Exception { assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 60L); //set topic-level policy, which should take effect - DispatchRate topicRate = new DispatchRate(10, 20L, 30); + DispatchRate topicRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(20) + .ratePeriodInSecond(30) + .build(); admin1.topics().setReplicatorDispatchRate(topicName, topicRate); Awaitility.await().untilAsserted(() -> assertEquals(admin1.topics().getReplicatorDispatchRate(topicName), topicRate)); @@ -122,7 +130,11 @@ public void testReplicatorRatePriority() throws Exception { assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 20L); //Set the namespace-level policy, which should not take effect - DispatchRate nsDispatchRate2 = new DispatchRate(500, 600L, 700); + DispatchRate nsDispatchRate2 = DispatchRate.builder() + .dispatchThrottlingRateInMsg(500) + .dispatchThrottlingRateInByte(600) + .ratePeriodInSecond(700) + .build(); admin1.namespaces().setReplicatorDispatchRate(namespace, nsDispatchRate2); Awaitility.await() .untilAsserted(() -> assertEquals(admin1.namespaces().getReplicatorDispatchRate(namespace), nsDispatchRate2)); @@ -180,7 +192,11 @@ public void testReplicatorRateLimiterDynamicallyChange() throws Exception { // 2. change namespace setting of replicator dispatchRateMsg, verify topic changed. int messageRate = 100; - DispatchRate dispatchRateMsg = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRateMsg = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); admin1.namespaces().setReplicatorDispatchRate(namespace, dispatchRateMsg); boolean replicatorUpdated = false; @@ -200,7 +216,11 @@ public void testReplicatorRateLimiterDynamicallyChange() throws Exception { // 3. change namespace setting of replicator dispatchRateByte, verify topic changed. messageRate = 500; - DispatchRate dispatchRateByte = new DispatchRate(-1, messageRate, 360); + DispatchRate dispatchRateByte = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(messageRate) + .ratePeriodInSecond(360) + .build(); admin1.namespaces().setReplicatorDispatchRate(namespace, dispatchRateByte); replicatorUpdated = false; for (int i = 0; i < retry; i++) { @@ -239,9 +259,17 @@ public void testReplicatorRateLimiterMessageNotReceivedAllMessages(DispatchRateT final int messageRate = 100; DispatchRate dispatchRate; if (DispatchRateType.messageRate.equals(dispatchRateType)) { - dispatchRate = new DispatchRate(messageRate, -1, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); } else { - dispatchRate = new DispatchRate(-1, messageRate, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(messageRate) + .ratePeriodInSecond(360) + .build(); } admin1.namespaces().setReplicatorDispatchRate(namespace, dispatchRate); @@ -323,7 +351,11 @@ public void testReplicatorRateLimiterMessageReceivedAllMessages() throws Excepti admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); final int messageRate = 100; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); admin1.namespaces().setReplicatorDispatchRate(namespace, dispatchRate); @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index fd544f266fc50..ead91a5af62a1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -71,6 +71,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.RetentionPolicy; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ReplicatorStats; import org.apache.pulsar.common.protocol.Commands; @@ -206,9 +207,9 @@ public Void call() throws Exception { public void activeBrokerParse() throws Exception { pulsar1.getConfiguration().setAuthorizationEnabled(true); //init clusterData - ClusterDataImpl cluster2Data = new ClusterDataImpl(); + String cluster2ServiceUrls = String.format("%s,localhost:1234,localhost:5678", pulsar2.getWebServiceAddress()); - cluster2Data.setServiceUrl(cluster2ServiceUrls); + ClusterData cluster2Data = ClusterData.builder().serviceUrl(cluster2ServiceUrls).build(); String cluster2 = "activeCLuster2"; admin2.clusters().createCluster(cluster2, cluster2Data); Awaitility.await().until(() @@ -492,7 +493,7 @@ public void testReplicatorClearBacklog() throws Exception { replicator.updateRates(); // for code-coverage replicator.expireMessages(1); // for code-coverage ReplicatorStats status = replicator.getStats(); - assertEquals(status.replicationBacklog, 0); + assertEquals(status.getReplicationBacklog(), 0); } @Test(timeOut = 30000) @@ -658,7 +659,10 @@ public void testResumptionAfterBacklogRelaxed() throws Exception { for (RetentionPolicy policy : policies) { // Use 1Mb quota by default - admin1.namespaces().setBacklogQuota("pulsar/ns1", new BacklogQuota(1 * 1024 * 1024, policy)); + admin1.namespaces().setBacklogQuota("pulsar/ns1", BacklogQuota.builder() + .limitSize(1 * 1024 * 1024) + .retentionPolicy(policy) + .build()); Thread.sleep(200); TopicName dest = TopicName @@ -683,7 +687,10 @@ public void testResumptionAfterBacklogRelaxed() throws Exception { Thread.sleep(500); // Restrict backlog quota limit to 1 byte to stop replication - admin1.namespaces().setBacklogQuota("pulsar/ns1", new BacklogQuota(1, policy)); + admin1.namespaces().setBacklogQuota("pulsar/ns1", BacklogQuota.builder() + .limitSize(1) + .retentionPolicy(policy) + .build()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index d81eb1e6ed39f..810d8ed1c61ce 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -36,6 +36,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.tests.TestRetrySupport; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; @@ -156,12 +157,24 @@ protected void setup() throws Exception { admin3 = PulsarAdmin.builder().serviceHttpUrl(url3.toString()).build(); // Provision the global namespace - admin1.clusters().createCluster("r1", new ClusterDataImpl(url1.toString(), urlTls1.toString(), - pulsar1.getSafeBrokerServiceUrl(), pulsar1.getBrokerServiceUrlTls())); - admin1.clusters().createCluster("r2", new ClusterDataImpl(url2.toString(), urlTls2.toString(), - pulsar2.getSafeBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls())); - admin1.clusters().createCluster("r3", new ClusterDataImpl(url3.toString(), urlTls3.toString(), - pulsar3.getSafeBrokerServiceUrl(), pulsar3.getBrokerServiceUrlTls())); + admin1.clusters().createCluster("r1", ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getSafeBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .build()); + admin1.clusters().createCluster("r2", ClusterData.builder() + .serviceUrl(url2.toString()) + .serviceUrlTls(urlTls2.toString()) + .brokerServiceUrl(pulsar2.getSafeBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) + .build()); + admin1.clusters().createCluster("r3", ClusterData.builder() + .serviceUrl(url3.toString()) + .serviceUrlTls(urlTls3.toString()) + .brokerServiceUrl(pulsar3.getSafeBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar3.getBrokerServiceUrlTls()) + .build()); admin1.tenants().createTenant("pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r2", "r3"))); @@ -176,7 +189,10 @@ protected void setup() throws Exception { assertEquals(admin2.clusters().getCluster("r3").getBrokerServiceUrl(), pulsar3.getSafeBrokerServiceUrl()); // Also create V1 namespace for compatibility check - admin1.clusters().createCluster("global", new ClusterDataImpl("http://global:8080", "https://global:8443")); + admin1.clusters().createCluster("global", ClusterData.builder() + .serviceUrl("http://global:8080") + .serviceUrlTls("https://global:8443") + .build()); admin1.namespaces().createNamespace("pulsar/global/ns"); admin1.namespaces().setNamespaceReplicationClusters("pulsar/global/ns", Sets.newHashSet("r1", "r2", "r3")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index 79715c23cdb0b..0d814594f85e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -25,6 +25,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicPolicies; @@ -205,7 +206,7 @@ public void testCacheCleanup() throws Exception { } private void prepareData() throws PulsarAdminException { - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("system-topic", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet("test"))); admin.namespaces().createNamespace(NAMESPACE1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java index 28cc9da98acf9..d55e1e7efe321 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java @@ -46,8 +46,9 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.metadata.api.extended.SessionEvent; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; @@ -117,9 +118,10 @@ void setup() throws Exception { leaderAdmin = pulsarAdmins[0]; Thread.sleep(1000); - pulsarAdmins[0].clusters().createCluster(testCluster, new ClusterDataImpl(pulsarServices[0].getWebServiceAddress())); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Sets.newHashSet(testCluster)); + pulsarAdmins[0].clusters().createCluster(testCluster, ClusterData.builder().serviceUrl(pulsarServices[0].getWebServiceAddress()).build()); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Sets.newHashSet(testCluster)) + .build(); pulsarAdmins[0].tenants().createTenant(testTenant, tenantInfo); pulsarAdmins[0].namespaces().createNamespace(testNamespace, 16); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java index 24e52f57dbe66..d43b046b9cdce 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java @@ -64,10 +64,10 @@ public void testAccumulativeStats() throws Exception { // stats are at zero before any activity TopicStats stats = topic.getStats(false, false); - assertEquals(stats.bytesInCounter, 0); - assertEquals(stats.msgInCounter, 0); - assertEquals(stats.bytesOutCounter, 0); - assertEquals(stats.msgOutCounter, 0); + assertEquals(stats.getBytesInCounter(), 0); + assertEquals(stats.getMsgInCounter(), 0); + assertEquals(stats.getBytesOutCounter(), 0); + assertEquals(stats.getMsgOutCounter(), 0); producer.newMessage().value("test").eventTime(5).send(); producer.newMessage().value("test").eventTime(5).send(); @@ -79,10 +79,10 @@ public void testAccumulativeStats() throws Exception { // send/receive result in non-zero stats TopicStats statsBeforeUnsubscribe = topic.getStats(false, false); - assertTrue(statsBeforeUnsubscribe.bytesInCounter > 0); - assertTrue(statsBeforeUnsubscribe.msgInCounter > 0); - assertTrue(statsBeforeUnsubscribe.bytesOutCounter > 0); - assertTrue(statsBeforeUnsubscribe.msgOutCounter > 0); + assertTrue(statsBeforeUnsubscribe.getBytesInCounter() > 0); + assertTrue(statsBeforeUnsubscribe.getMsgInCounter() > 0); + assertTrue(statsBeforeUnsubscribe.getBytesOutCounter() > 0); + assertTrue(statsBeforeUnsubscribe.getMsgOutCounter() > 0); consumer1.unsubscribe(); consumer2.unsubscribe(); @@ -92,9 +92,9 @@ public void testAccumulativeStats() throws Exception { // consumer unsubscribe/producer removal does not result in stats loss TopicStats statsAfterUnsubscribe = topic.getStats(false, false); - assertEquals(statsAfterUnsubscribe.bytesInCounter, statsBeforeUnsubscribe.bytesInCounter); - assertEquals(statsAfterUnsubscribe.msgInCounter, statsBeforeUnsubscribe.msgInCounter); - assertEquals(statsAfterUnsubscribe.bytesOutCounter, statsBeforeUnsubscribe.bytesOutCounter); - assertEquals(statsAfterUnsubscribe.msgOutCounter, statsBeforeUnsubscribe.msgOutCounter); + assertEquals(statsAfterUnsubscribe.getBytesInCounter(), statsBeforeUnsubscribe.getBytesInCounter()); + assertEquals(statsAfterUnsubscribe.getMsgInCounter(), statsBeforeUnsubscribe.getMsgInCounter()); + assertEquals(statsAfterUnsubscribe.getBytesOutCounter(), statsBeforeUnsubscribe.getBytesOutCounter()); + assertEquals(statsAfterUnsubscribe.getMsgOutCounter(), statsBeforeUnsubscribe.getMsgOutCounter()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 01cea74d4168a..bca6342a93a27 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -335,7 +335,10 @@ public void testEnableAndDisableTopicDelayedDelivery() throws Exception { admin.topics().createPartitionedTopic(topicName, 3); assertNull(admin.topics().getDelayedDeliveryPolicy(topicName)); - DelayedDeliveryPolicies delayedDeliveryPolicies = new DelayedDeliveryPolicies(2000, false); + DelayedDeliveryPolicies delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(2000) + .active(false) + .build();; admin.topics().setDelayedDeliveryPolicy(topicName, delayedDeliveryPolicies); //wait for update for (int i = 0; i < 50; i++) { @@ -366,7 +369,10 @@ public void testEnableTopicDelayedDelivery() throws Exception { admin.topics().createPartitionedTopic(topicName, 3); assertNull(admin.topics().getDelayedDeliveryPolicy(topicName)); //1 Set topic policy - DelayedDeliveryPolicies delayedDeliveryPolicies = new DelayedDeliveryPolicies(2000, true); + DelayedDeliveryPolicies delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(2000) + .active(true) + .build(); admin.topics().setDelayedDeliveryPolicy(topicName, delayedDeliveryPolicies); //wait for update for (int i = 0; i < 50; i++) { @@ -408,7 +414,10 @@ public void testEnableTopicDelayedDelivery() throws Exception { assertTrue(delayedMessages.contains("delayed-msg-" + i)); } //5 Disable delayed delivery - delayedDeliveryPolicies.setActive(false); + delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(2000) + .active(false) + .build(); admin.topics().setDelayedDeliveryPolicy(topicName, delayedDeliveryPolicies); //wait for update for (int i = 0; i < 50; i++) { @@ -423,8 +432,10 @@ public void testEnableTopicDelayedDelivery() throws Exception { assertNotNull(msg); consumer.acknowledge(msg); //7 Set a very long tick time, so that trackDelayedDelivery will fail. we can receive msg immediately. - delayedDeliveryPolicies.setActive(true); - delayedDeliveryPolicies.setTickTime(Integer.MAX_VALUE); + delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .tickTime(Integer.MAX_VALUE) + .active(true) + .build(); admin.topics().setDelayedDeliveryPolicy(topicName, delayedDeliveryPolicies); //wait for update for (int i = 0; i < 50; i++) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java index b97b53aeeb087..884d2a0f4e8d3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java @@ -30,7 +30,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; @@ -41,8 +42,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; import org.apache.bookkeeper.common.util.OrderedExecutor; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; @@ -60,7 +59,6 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.PersistentTopicTest; import org.apache.pulsar.broker.transaction.buffer.impl.InMemTransactionBufferProvider; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 4705dd71c0535..14590a4f55f69 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -207,10 +207,10 @@ public void testAccumulativeStats() throws Exception { // stats are at zero before any activity TopicStats stats = topic.getStats(false, false); - assertEquals(stats.bytesInCounter, 0); - assertEquals(stats.msgInCounter, 0); - assertEquals(stats.bytesOutCounter, 0); - assertEquals(stats.msgOutCounter, 0); + assertEquals(stats.getBytesInCounter(), 0); + assertEquals(stats.getMsgInCounter(), 0); + assertEquals(stats.getBytesOutCounter(), 0); + assertEquals(stats.getMsgOutCounter(), 0); producer.newMessage().value("test").eventTime(5).send(); producer.newMessage().value("test").eventTime(5).send(); @@ -222,10 +222,10 @@ public void testAccumulativeStats() throws Exception { // send/receive result in non-zero stats TopicStats statsBeforeUnsubscribe = topic.getStats(false, false); - assertTrue(statsBeforeUnsubscribe.bytesInCounter > 0); - assertTrue(statsBeforeUnsubscribe.msgInCounter > 0); - assertTrue(statsBeforeUnsubscribe.bytesOutCounter > 0); - assertTrue(statsBeforeUnsubscribe.msgOutCounter > 0); + assertTrue(statsBeforeUnsubscribe.getBytesInCounter() > 0); + assertTrue(statsBeforeUnsubscribe.getMsgInCounter() > 0); + assertTrue(statsBeforeUnsubscribe.getBytesOutCounter() > 0); + assertTrue(statsBeforeUnsubscribe.getMsgOutCounter() > 0); consumer1.unsubscribe(); consumer2.unsubscribe(); @@ -235,9 +235,9 @@ public void testAccumulativeStats() throws Exception { // consumer unsubscribe/producer removal does not result in stats loss TopicStats statsAfterUnsubscribe = topic.getStats(false, false); - assertEquals(statsAfterUnsubscribe.bytesInCounter, statsBeforeUnsubscribe.bytesInCounter); - assertEquals(statsAfterUnsubscribe.msgInCounter, statsBeforeUnsubscribe.msgInCounter); - assertEquals(statsAfterUnsubscribe.bytesOutCounter, statsBeforeUnsubscribe.bytesOutCounter); - assertEquals(statsAfterUnsubscribe.msgOutCounter, statsBeforeUnsubscribe.msgOutCounter); + assertEquals(statsAfterUnsubscribe.getBytesInCounter(), statsBeforeUnsubscribe.getBytesInCounter()); + assertEquals(statsAfterUnsubscribe.getMsgInCounter(), statsBeforeUnsubscribe.getMsgInCounter()); + assertEquals(statsAfterUnsubscribe.getBytesOutCounter(), statsBeforeUnsubscribe.getBytesOutCounter()); + assertEquals(statsAfterUnsubscribe.getMsgOutCounter(), statsBeforeUnsubscribe.getMsgOutCounter()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java index 074dfca286f56..aa513661f1bd5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java @@ -61,13 +61,13 @@ public void createReplicatedSubscription() throws Exception { .subscribe(); TopicStats stats = admin.topics().getStats(topic); - assertTrue(stats.subscriptions.get("sub1").isReplicated); + assertTrue(stats.getSubscriptions().get("sub1").isReplicated()); admin.topics().unload(topic); // Check that subscription is still marked replicated after reloading stats = admin.topics().getStats(topic); - assertTrue(stats.subscriptions.get("sub1").isReplicated); + assertTrue(stats.getSubscriptions().get("sub1").isReplicated()); } @Test @@ -82,7 +82,7 @@ public void upgradeToReplicatedSubscription() throws Exception { .subscribe(); TopicStats stats = admin.topics().getStats(topic); - assertFalse(stats.subscriptions.get("sub").isReplicated); + assertFalse(stats.getSubscriptions().get("sub").isReplicated()); consumer.close(); consumer = pulsarClient.newConsumer(Schema.STRING) @@ -92,7 +92,7 @@ public void upgradeToReplicatedSubscription() throws Exception { .subscribe(); stats = admin.topics().getStats(topic); - assertTrue(stats.subscriptions.get("sub").isReplicated); + assertTrue(stats.getSubscriptions().get("sub").isReplicated()); consumer.close(); } @@ -108,7 +108,7 @@ public void upgradeToReplicatedSubscriptionAfterRestart() throws Exception { .subscribe(); TopicStats stats = admin.topics().getStats(topic); - assertFalse(stats.subscriptions.get("sub").isReplicated); + assertFalse(stats.getSubscriptions().get("sub").isReplicated()); consumer.close(); admin.topics().unload(topic); @@ -120,7 +120,7 @@ public void upgradeToReplicatedSubscriptionAfterRestart() throws Exception { .subscribe(); stats = admin.topics().getStats(topic); - assertTrue(stats.subscriptions.get("sub").isReplicated); + assertTrue(stats.getSubscriptions().get("sub").isReplicated()); consumer.close(); } @@ -135,7 +135,7 @@ public void testDisableReplicatedSubscriptions() throws Exception { .subscribe(); TopicStats stats = admin.topics().getStats(topic); - assertFalse(stats.subscriptions.get("sub").isReplicated); + assertFalse(stats.getSubscriptions().get("sub").isReplicated()); consumer.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 63abcf33e16c0..c30024974efa2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -86,10 +87,10 @@ public void testConsumerStatsOnZeroMaxUnackedMessagesPerConsumer() throws Pulsar received = 0; TopicStats stats = admin.topics().getStats(topicName); - Assert.assertEquals(stats.subscriptions.size(), 1); - Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue().consumers.size(), 1); - Assert.assertFalse(stats.subscriptions.entrySet().iterator().next().getValue().consumers.get(0).blockedConsumerOnUnackedMsgs); - Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue().consumers.get(0).unackedMessages, messages); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().size(), 1); + Assert.assertFalse(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().get(0).isBlockedConsumerOnUnackedMsgs()); + Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().get(0).getUnackedMessages(), messages); for (int i = 0; i < messages; i++) { consumer.acknowledge(consumer.receive()); @@ -103,8 +104,8 @@ public void testConsumerStatsOnZeroMaxUnackedMessagesPerConsumer() throws Pulsar stats = admin.topics().getStats(topicName); - Assert.assertFalse(stats.subscriptions.entrySet().iterator().next().getValue().consumers.get(0).blockedConsumerOnUnackedMsgs); - Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue().consumers.get(0).unackedMessages, 0); + Assert.assertFalse(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().get(0).isBlockedConsumerOnUnackedMsgs()); + Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().get(0).getUnackedMessages(), 0); } @Test @@ -138,9 +139,9 @@ public void testAckStatsOnPartitionedTopicForExclusiveSubscription() throws Puls for (int i = 0; i < 3; i++) { TopicStats stats = admin.topics().getStats(topic + "-partition-" + i); - Assert.assertEquals(stats.subscriptions.size(), 1); - Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue().consumers.size(), 1); - Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue().consumers.get(0).unackedMessages, 0); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().size(), 1); + Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue().getConsumers().get(0).getUnackedMessages(), 0); } } @@ -159,13 +160,13 @@ public void testUpdateStatsForActiveConsumerAndSubscription() throws Exception { List consumers = topicRef.getSubscriptions() .get("my-subscription").getConsumers(); Assert.assertEquals(consumers.size(), 1); - ConsumerStats consumerStats = new ConsumerStats(); + ConsumerStatsImpl consumerStats = new ConsumerStatsImpl(); consumerStats.msgOutCounter = 10; consumerStats.bytesOutCounter = 1280; consumers.get(0).updateStats(consumerStats); ConsumerStats updatedStats = consumers.get(0).getStats(); - Assert.assertEquals(updatedStats.msgOutCounter, 10); - Assert.assertEquals(updatedStats.bytesOutCounter, 1280); + Assert.assertEquals(updatedStats.getMsgOutCounter(), 10); + Assert.assertEquals(updatedStats.getBytesOutCounter(), 1280); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index d5516cd5b8151..f64edea1544b2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -87,9 +87,9 @@ public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarA .subscribe(); TopicStats stats = admin.topics().getStats(topicName); - Assert.assertEquals(stats.subscriptions.size(), 1); - Assert.assertEquals(stats.subscriptions.entrySet().iterator().next().getValue() - .consumersAfterMarkDeletePosition.size(), 1); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue() + .getConsumersAfterMarkDeletePosition().size(), 1); consumer1.close(); consumer2.close(); @@ -127,12 +127,12 @@ public void testNonContiguousDeletedMessagesRanges() throws Exception { Awaitility.await().untilAsserted(() -> { TopicStats stats = admin.topics().getStats(topicName); - Assert.assertEquals(stats.nonContiguousDeletedMessagesRanges, 1); - Assert.assertEquals(stats.subscriptions.size(), 1); - Assert.assertEquals(stats.subscriptions.get(subName).nonContiguousDeletedMessagesRanges, 1); - Assert.assertTrue(stats.nonContiguousDeletedMessagesRangesSerializedSize > 0); - Assert.assertTrue(stats.subscriptions.get(subName) - .nonContiguousDeletedMessagesRangesSerializedSize > 0); + Assert.assertEquals(stats.getNonContiguousDeletedMessagesRanges(), 1); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + Assert.assertEquals(stats.getSubscriptions().get(subName).getNonContiguousDeletedMessagesRanges(), 1); + Assert.assertTrue(stats.getNonContiguousDeletedMessagesRangesSerializedSize() > 0); + Assert.assertTrue(stats.getSubscriptions().get(subName) + .getNonContiguousDeletedMessagesRangesSerializedSize() > 0); }); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 16a43b664b765..cd053062ffdfe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -34,7 +34,6 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionSubscription; import org.apache.pulsar.transaction.coordinator.TxnMeta; @@ -198,7 +197,10 @@ public void testManagedLedgerMetrics() throws Exception{ String subName = "test_managed_ledger_metrics"; admin.topics().createNonPartitionedTopic(topic); admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(), - new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); + TenantInfo.builder() + .adminRoles(Sets.newHashSet("appid1")) + .allowedClusters(Sets.newHashSet("test")) + .build()); admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString()); admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1); TransactionCoordinatorID transactionCoordinatorIDOne = TransactionCoordinatorID.get(0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java index daaf4a3444b72..b524e1a2148c6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java @@ -29,6 +29,7 @@ import org.apache.pulsar.common.events.TopicPoliciesEvent; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicPolicies; @@ -123,7 +124,7 @@ public void checkSystemTopic() throws PulsarAdminException { } private void prepareData() throws PulsarAdminException { - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("system-topic", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet("test"))); admin.namespaces().createNamespace(NAMESPACE1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 2078b265123d3..f8e0e2a5d8b6e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -53,6 +53,7 @@ import org.apache.pulsar.common.events.EventsTopicNames; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -85,7 +86,7 @@ protected void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NAMESPACE1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionClientReconnectTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionClientReconnectTest.java index 9c5c36614e776..2a336f6a4f6ec 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionClientReconnectTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionClientReconnectTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; @@ -53,7 +54,7 @@ public void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant("public", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace("public/txn", 10); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java index 1d0b28e9d805c..bfa4284f235aa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.common.api.proto.MessageIdData; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.TxnAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.protocol.Commands; @@ -69,7 +70,7 @@ public void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant("public", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace("public/txn", 10); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java index 6b69bf1899ec1..cc4e8e249a4bf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.protocol.Commands; @@ -87,7 +88,7 @@ protected void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NAMESPACE1); 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 5fb58ae42b284..5d088bbacd5a6 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 @@ -54,6 +54,7 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.api.proto.TxnAction; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -85,7 +86,7 @@ public class TransactionBufferClientTest extends TransactionMetaStoreTestBase { @Override protected void afterSetup() throws Exception { - pulsarAdmins[0].clusters().createCluster("my-cluster", new ClusterDataImpl(pulsarServices[0].getWebServiceAddress())); + pulsarAdmins[0].clusters().createCluster("my-cluster", ClusterData.builder().serviceUrl(pulsarServices[0].getWebServiceAddress()).build()); pulsarAdmins[0].tenants().createTenant("public", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet("my-cluster"))); pulsarAdmins[0].namespaces().createNamespace(namespace, 10); pulsarAdmins[0].topics().createPartitionedTopic(partitionedTopicName.getPartitionedTopicName(), partitions); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java index 20d92fb0c7bb7..a9fc325a58246 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -86,7 +87,7 @@ protected void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NAMESPACE1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java index 3c91884ebd67d..0e9bf75e858f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java @@ -39,6 +39,7 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; @@ -63,7 +64,7 @@ protected void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NAMESPACE1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java index 5ed73f3c47261..fd5a2b8e5e068 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.collections.BitSetRecyclable; @@ -80,7 +81,7 @@ protected void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 3143916963270..191e40276f4a4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -68,7 +68,7 @@ public void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterDataImpl.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index ea99572a8d5ed..48bf6025675ec 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -63,7 +63,9 @@ import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.PulsarAdminException.ConflictException; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.SecurityUtility; @@ -272,8 +274,9 @@ public void testMaxRequestSize() throws Exception { .setHeader("Content-Type", "application/json"); // HTTP server is configured to reject everything > 10K - TenantInfoImpl info1 = new TenantInfoImpl(); - info1.setAdminRoles(Collections.singleton(StringUtils.repeat("*", 20 * 1024))); + TenantInfo info1 = TenantInfo.builder() + .adminRoles(Collections.singleton(StringUtils.repeat("*", 20 * 1024))) + .build(); builder.setBody(ObjectMapperFactory.getThreadLocal().writeValueAsBytes(info1)); Response res = builder.execute().get(); @@ -283,10 +286,11 @@ public void testMaxRequestSize() throws Exception { // Create local cluster String localCluster = "test"; String clusterPath = PulsarWebResource.path("clusters", localCluster); - pulsar.getPulsarResources().getClusterResources().create(clusterPath, new ClusterDataImpl()); - TenantInfoImpl info2 = new TenantInfoImpl(); - info2.setAdminRoles(Collections.singleton(StringUtils.repeat("*", 1 * 1024))); - info2.setAllowedClusters(Sets.newHashSet(localCluster)); + pulsar.getPulsarResources().getClusterResources().create(clusterPath, ClusterDataImpl.builder().build()); + TenantInfo info2 = TenantInfo.builder() + .adminRoles(Collections.singleton(StringUtils.repeat("*", 1 * 1024))) + .allowedClusters(Sets.newHashSet(localCluster)) + .build(); builder.setBody(ObjectMapperFactory.getThreadLocal().writeValueAsBytes(info2)); Response res2 = builder.execute().get(); @@ -434,7 +438,7 @@ public CompletableFuture create(String serverList, SessionType sessio try { pulsarAdmin.clusters().createCluster(config.getClusterName(), - new ClusterDataImpl(pulsar.getSafeWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); } catch (ConflictException ce) { // This is OK. } finally { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperSessionExpireRecoveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperSessionExpireRecoveryTest.java index acd631ed71c6e..98b8085236589 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperSessionExpireRecoveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperSessionExpireRecoveryTest.java @@ -25,6 +25,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.MockZooKeeper; @@ -52,7 +53,7 @@ protected void cleanup() throws Exception { */ @Test public void testSessionExpired() throws Exception { - admin.clusters().createCluster("my-cluster", new ClusterDataImpl("test-url")); + admin.clusters().createCluster("my-cluster", ClusterData.builder().serviceUrl("test-url").build()); assertTrue(Sets.newHashSet(admin.clusters().getClusters()).contains("my-cluster")); @@ -64,12 +65,12 @@ public void testSessionExpired() throws Exception { assertTrue(Sets.newHashSet(admin.clusters().getClusters()).contains("my-cluster")); try { - admin.clusters().createCluster("my-cluster-2", new ClusterDataImpl("test-url")); + admin.clusters().createCluster("my-cluster-2", ClusterData.builder().serviceUrl("test-url").build()); fail("Should have failed, because global zk is down"); } catch (PulsarAdminException e) { // Ok } - admin.clusters().createCluster("cluster-2", new ClusterDataImpl("test-url")); + admin.clusters().createCluster("cluster-2", ClusterData.builder().serviceUrl("test-url").build()); } } 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 86445b626eb67..6d76ce8cbfe8e 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 @@ -37,6 +37,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationBasic; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.zookeeper.KeeperException.Code; @@ -171,7 +172,7 @@ public void testTlsSyncProducerAndConsumer(int batchMessageDelayMs) throws Excep authTls.configure(authParams); internalSetup(authTls); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); @@ -189,7 +190,7 @@ public void testBasicCryptSyncProducerAndConsumer(int batchMessageDelayMs) throw authPassword.configure("{\"userId\":\"superUser\",\"password\":\"supepass\"}"); internalSetup(authPassword); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet("test"))); @@ -207,7 +208,7 @@ public void testBasicArp1SyncProducerAndConsumer(int batchMessageDelayMs) throws authPassword.configure("{\"userId\":\"superUser2\",\"password\":\"superpassword\"}"); internalSetup(authPassword); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet("test"))); @@ -229,8 +230,13 @@ public void testAnonymousSyncProducerAndConsumer(int batchMessageDelayMs) throws authTls.configure(authParams); internalSetup(authTls); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster("test", + ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("anonymousUser"), Sets.newHashSet("test"))); @@ -270,8 +276,12 @@ public void testAuthenticationFilterNegative() throws Exception { internalSetup(authTls); final String cluster = "test"; - final ClusterDataImpl clusterData = new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + final ClusterData clusterData = ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build(); try { admin.clusters().createCluster(cluster, clusterData); } catch (PulsarAdminException e) { @@ -298,8 +308,12 @@ public void testInternalServerExceptionOnLookup() throws Exception { authTls.configure(authParams); internalSetup(authTls); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster("test", ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); String namespace = "my-property/my-ns"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java index 75ff745ce9f0a..bb8a02143e5e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.tls.PublicSuffixMatcher; import org.apache.pulsar.common.tls.TlsHostnameVerifier; import org.apache.pulsar.common.policies.data.ClusterDataImpl; @@ -117,7 +118,9 @@ protected void setupClient() throws Exception { .tlsTrustCertsFilePath(TLS_MIM_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(true) .authentication(authTls).enableTls(true).enableTlsHostnameVerification(hostnameVerificationEnabled)); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java index 47c23bbef8daf..261732a22d7a1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java @@ -48,8 +48,10 @@ import org.apache.pulsar.common.naming.NamespaceName; 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.ClusterDataImpl; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.policies.data.TopicOperation; @@ -129,7 +131,7 @@ public void testProducerAndConsumerAuthorization() throws Exception { .operationTimeout(1000, TimeUnit.MILLISECONDS) .authentication(authenticationInvalidRole).build(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); @@ -194,7 +196,7 @@ public void testSubscriberPermission() throws Exception { Authentication authentication = new ClientAuthentication(subscriptionRole); - superAdmin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + superAdmin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); superAdmin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet(tenantRole), Sets.newHashSet("test"))); @@ -291,7 +293,7 @@ public void testSubscriptionPrefixAuthorization() throws Exception { .authentication(authentication)); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("prop-prefix", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); @@ -512,7 +514,7 @@ public CompletableFuture revokeSubscriptionPermissionAsync(NamespaceName n } @Override - public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfoImpl tenantInfo, AuthenticationDataSource authenticationData) { + public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfo tenantInfo, AuthenticationDataSource authenticationData) { return CompletableFuture.completedFuture(true); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 8fa8d45f159d5..cb349bf22783c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -83,6 +83,7 @@ import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -271,7 +272,10 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { String broker2ServiceUrl = "pulsar://localhost:" + conf2.getBrokerServicePort().get(); admin.clusters().createCluster(newCluster, - new ClusterDataImpl(pulsar.getWebServiceAddress(), null, broker2ServiceUrl, null)); + ClusterData.builder() + .serviceUrl(pulsar.getWebServiceAddress()) + .brokerServiceUrl(broker2ServiceUrl) + .build()); admin.tenants().createTenant(property, new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet(newCluster))); admin.namespaces().createNamespace(property + "/" + newCluster + "/my-ns"); @@ -1085,7 +1089,7 @@ public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { final int totalPartitions = 10; final TopicName dest = TopicName.get("persistent", property, cluster, namespace, topicName); admin.clusters().createCluster(cluster, - new ClusterDataImpl(pulsar.getWebServiceAddress(), null, null, null)); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant(property, new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet(cluster))); admin.namespaces().createNamespace(property + "/" + cluster + "/" + namespace); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java index b28c39b88b3fd..538bb895b524a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.Sets; import java.lang.reflect.Method; import java.net.URL; +import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.Optional; @@ -46,8 +47,10 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; @@ -106,7 +109,7 @@ void setup(Method method) throws Exception { primaryHost = pulsar.getWebServiceAddress(); // update cluster metadata - ClusterDataImpl clusterData = new ClusterDataImpl(url.toString()); + ClusterData clusterData = ClusterData.builder().serviceUrl(url.toString()).build(); admin.clusters().createCluster(config.getClusterName(), clusterData); if (pulsarClient != null) { @@ -115,8 +118,9 @@ void setup(Method method) throws Exception { ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).maxBackoffInterval(1, TimeUnit.SECONDS); pulsarClient = clientBuilder.build(); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton("use")) + .build(); admin.tenants().createTenant(tenant, tenantInfo); } @@ -207,16 +211,16 @@ public void testClientDeduplicationCorrectnessWithFailure() throws Exception { retryStrategically((test) -> { try { TopicStats topicStats = admin.topics().getStats(sourceTopic); - return topicStats.publishers.size() == 1 && topicStats.publishers.get(0).getProducerName().equals("test-producer-1") && topicStats.storageSize > 0; + return topicStats.getPublishers().size() == 1 && topicStats.getPublishers().get(0).getProducerName().equals("test-producer-1") && topicStats.getStorageSize() > 0; } catch (PulsarAdminException e) { return false; } }, 5, 200); TopicStats topicStats = admin.topics().getStats(sourceTopic); - assertEquals(topicStats.publishers.size(), 1); - assertEquals(topicStats.publishers.get(0).getProducerName(), "test-producer-1"); - assertTrue(topicStats.storageSize > 0); + assertEquals(topicStats.getPublishers().size(), 1); + assertEquals(topicStats.getPublishers().get(0).getProducerName(), "test-producer-1"); + assertTrue(topicStats.getStorageSize() > 0); for (int i = 0; i < 5; i++) { log.info("Stopping BK..."); @@ -303,14 +307,14 @@ public void testClientDeduplicationWithBkFailure() throws Exception { try { TopicStats topicStats = admin.topics().getStats(sourceTopic); boolean c1 = topicStats!= null - && topicStats.subscriptions.get(subscriptionName1) != null - && topicStats.subscriptions.get(subscriptionName1).consumers.size() == 1 - && topicStats.subscriptions.get(subscriptionName1).consumers.get(0).consumerName.equals(consumerName1); + && topicStats.getSubscriptions().get(subscriptionName1) != null + && topicStats.getSubscriptions().get(subscriptionName1).getConsumers().size() == 1 + && topicStats.getSubscriptions().get(subscriptionName1).getConsumers().get(0).getConsumerName().equals(consumerName1); boolean c2 = topicStats!= null - && topicStats.subscriptions.get(subscriptionName2) != null - && topicStats.subscriptions.get(subscriptionName2).consumers.size() == 1 - && topicStats.subscriptions.get(subscriptionName2).consumers.get(0).consumerName.equals(consumerName2); + && topicStats.getSubscriptions().get(subscriptionName2) != null + && topicStats.getSubscriptions().get(subscriptionName2).getConsumers().size() == 1 + && topicStats.getSubscriptions().get(subscriptionName2).getConsumers().get(0).getConsumerName().equals(consumerName2); return c1 && c2; } catch (PulsarAdminException e) { return false; @@ -319,14 +323,14 @@ public void testClientDeduplicationWithBkFailure() throws Exception { TopicStats topicStats1 = admin.topics().getStats(sourceTopic); assertNotNull(topicStats1); - assertNotNull(topicStats1.subscriptions.get(subscriptionName1)); - assertEquals(topicStats1.subscriptions.get(subscriptionName1).consumers.size(), 1); - assertEquals(topicStats1.subscriptions.get(subscriptionName1).consumers.get(0).consumerName, consumerName1); + assertNotNull(topicStats1.getSubscriptions().get(subscriptionName1)); + assertEquals(topicStats1.getSubscriptions().get(subscriptionName1).getConsumers().size(), 1); + assertEquals(topicStats1.getSubscriptions().get(subscriptionName1).getConsumers().get(0).getConsumerName(), consumerName1); TopicStats topicStats2 = admin.topics().getStats(sourceTopic); assertNotNull(topicStats2); - assertNotNull(topicStats2.subscriptions.get(subscriptionName2)); - assertEquals(topicStats2.subscriptions.get(subscriptionName2).consumers.size(), 1); - assertEquals(topicStats2.subscriptions.get(subscriptionName2).consumers.get(0).consumerName, consumerName2); + assertNotNull(topicStats2.getSubscriptions().get(subscriptionName2)); + assertEquals(topicStats2.getSubscriptions().get(subscriptionName2).getConsumers().size(), 1); + assertEquals(topicStats2.getSubscriptions().get(subscriptionName2).getConsumers().get(0).getConsumerName(), consumerName2); for (int i=0; i<10; i++) { producer.newMessage().sequenceId(i).value("foo-" + i).send(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java index 9900c32227cd3..d43a759bf3622 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java @@ -546,12 +546,12 @@ public void testBlockDispatcherStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); // subscription stats - assertEquals(stats.subscriptions.keySet().size(), 1); - assertEquals(subStats.msgBacklog, 0); - assertEquals(subStats.consumers.size(), 1); + assertEquals(stats.getSubscriptions().keySet().size(), 1); + assertEquals(subStats.getMsgBacklog(), 0); + assertEquals(subStats.getConsumers().size(), 1); Producer producer = pulsarClient.newProducer().topic(topicName).create(); Thread.sleep(timeWaitToSync); @@ -564,17 +564,17 @@ public void testBlockDispatcherStats() throws Exception { rolloverPerIntervalStats(); stats = topicRef.getStats(false, false); - subStats = stats.subscriptions.values().iterator().next(); + subStats = stats.getSubscriptions().values().iterator().next(); - assertTrue(subStats.msgBacklog > 0); - assertTrue(subStats.unackedMessages > 0); - assertTrue(subStats.blockedSubscriptionOnUnackedMsgs); - assertEquals(subStats.consumers.get(0).unackedMessages, subStats.unackedMessages); + assertTrue(subStats.getMsgBacklog() > 0); + assertTrue(subStats.getUnackedMessages() > 0); + assertTrue(subStats.isBlockedSubscriptionOnUnackedMsgs()); + assertEquals(subStats.getConsumers().get(0).getUnackedMessages(), subStats.getUnackedMessages()); // consumer stats - assertTrue(subStats.consumers.get(0).msgRateOut > 0.0); - assertTrue(subStats.consumers.get(0).msgThroughputOut > 0.0); - assertEquals(subStats.msgRateRedeliver, 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgRateOut() > 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgThroughputOut() > 0.0); + assertEquals(subStats.getMsgRateRedeliver(), 0.0); producer.close(); consumer.close(); 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 e58f21a5c4799..fc3cfaf3f9eee 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 @@ -38,9 +38,11 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; 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.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -124,7 +126,11 @@ public void testMessageRateDynamicallyChange() throws Exception { // (2) change to 100 int messageRate = 100; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); admin.namespaces().setDispatchRate(namespace, dispatchRate); boolean isDispatchRateUpdate = false; int retry = 5; @@ -148,7 +154,11 @@ public void testMessageRateDynamicallyChange() throws Exception { // (3) change to 500 messageRate = 500; - dispatchRate = new DispatchRate(-1, messageRate, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(messageRate) + .ratePeriodInSecond(360) + .build(); admin.namespaces().setDispatchRate(namespace, dispatchRate); isDispatchRateUpdate = false; for (int i = 0; i < retry; i++) { @@ -188,9 +198,17 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr final int messageRate = 100; DispatchRate dispatchRate = null; if (DispatchRateType.messageRate.equals(dispatchRateType)) { - dispatchRate = new DispatchRate(messageRate, -1, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); } else { - dispatchRate = new DispatchRate(-1, messageRate, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(messageRate) + .ratePeriodInSecond(360) + .build(); } admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); @@ -326,7 +344,11 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio final String topicName = "persistent://" + namespace + "/throttlingAll"; final int messageRate = 10; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(1) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic @@ -397,7 +419,11 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT final String topicName = "persistent://" + namespace + "/throttlingAll"; final int byteRate = 100; - DispatchRate dispatchRate = new DispatchRate(-1, byteRate, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(byteRate) + .ratePeriodInSecond(1) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic @@ -460,7 +486,11 @@ public void testRateLimitingMultipleConsumers() throws Exception { final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; final int messageRate = 5; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic @@ -593,7 +623,11 @@ public void testMessageByteRateThrottlingCombined(SubscriptionType subscription) final int messageRate = 5; // 5 msgs per second final long byteRate = 10; // 10 bytes per second - DispatchRate dispatchRate = new DispatchRate(messageRate, byteRate, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(byteRate) + .ratePeriodInSecond(360) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic @@ -666,9 +700,13 @@ public void testGlobalNamespaceThrottling() throws Exception { final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); - admin.clusters().createCluster("global", new ClusterDataImpl("http://global:8080")); + admin.clusters().createCluster("global", ClusterData.builder().serviceUrl("http://global:8080").build()); admin.namespaces().createNamespace(namespace); admin.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); @@ -736,7 +774,11 @@ public void testNonBacklogConsumerWithThrottlingEnabled(SubscriptionType subscri final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 10; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); @@ -834,7 +876,11 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { // (2) Update namespace throttling limit int nsMessageRate = 500; - DispatchRate dispatchRate = new DispatchRate(nsMessageRate, 0, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(nsMessageRate) + .dispatchThrottlingRateInByte(0) + .ratePeriodInSecond(1) + .build(); admin.namespaces().setDispatchRate(namespace, dispatchRate); for (int i = 0; i < 5; i++) { if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() != nsMessageRate) { @@ -844,7 +890,11 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { Assert.assertEquals(nsMessageRate, topic.getDispatchRateLimiter().get().getDispatchRateOnMsg()); // (3) Disable namespace throttling limit will force to take cluster-config - dispatchRate = new DispatchRate(0, 0, 1); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(0) + .dispatchThrottlingRateInByte(0) + .ratePeriodInSecond(1) + .build(); admin.namespaces().setDispatchRate(namespace, dispatchRate); for (int i = 0; i < 5; i++) { if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() == nsMessageRate) { @@ -872,7 +922,11 @@ public void testClosingRateLimiter(SubscriptionType subscription) throws Excepti final String topicName = "persistent://" + namespace + "/closingRateLimiter" + subscription.name(); final String subName = "mySubscription" + subscription.name(); - DispatchRate dispatchRate = new DispatchRate(10, 1024, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(1024) + .ratePeriodInSecond(1) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); @@ -915,11 +969,19 @@ public void testDispatchRateCompatibility1() throws Exception { final String cluster = "test"; Optional policies = Optional.of(new Policies()); - DispatchRate clusterDispatchRate = new DispatchRate(100, 512, 1); - DispatchRate topicDispatchRate = new DispatchRate(200, 1024, 1); + DispatchRateImpl clusterDispatchRate = DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(512) + .ratePeriodInSecond(1) + .build(); + DispatchRateImpl topicDispatchRate = DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(200) + .dispatchThrottlingRateInByte(1024) + .ratePeriodInSecond(1) + .build(); // (1) If both clusterDispatchRate and topicDispatchRate are empty, dispatch throttling is disabled - DispatchRate dispatchRate = DispatchRateLimiter.getPoliciesDispatchRate(cluster, policies, + DispatchRateImpl dispatchRate = DispatchRateLimiter.getPoliciesDispatchRate(cluster, policies, DispatchRateLimiter.Type.TOPIC); Assert.assertNull(dispatchRate); @@ -946,8 +1008,16 @@ public void testDispatchRateCompatibility2() throws Exception { DispatchRateLimiter dispatchRateLimiter = new DispatchRateLimiter(topic, DispatchRateLimiter.Type.TOPIC); Policies policies = new Policies(); - DispatchRate clusterDispatchRate = new DispatchRate(100, 512, 1); - DispatchRate topicDispatchRate = new DispatchRate(200, 1024, 1); + DispatchRateImpl clusterDispatchRate = DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(100) + .dispatchThrottlingRateInByte(512) + .ratePeriodInSecond(1) + .build(); + DispatchRateImpl topicDispatchRate = DispatchRateImpl.builder() + .dispatchThrottlingRateInMsg(200) + .dispatchThrottlingRateInByte(1024) + .ratePeriodInSecond(1) + .build(); // (1) If both clusterDispatchRate and topicDispatchRate are empty, dispatch throttling is disabled dispatchRateLimiter.onPoliciesUpdate(policies); @@ -995,7 +1065,12 @@ public void testRelativeMessageRateLimitingThrottling(SubscriptionType subscript final String topicName = "persistent://" + namespace + "/relative-throttle" + subscription; final int messageRate = 1; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 1, true); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(1) + .relativeToPublishRate(true) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java index a766b40e5cf21..1e4b1ded3dd29 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java @@ -59,6 +59,7 @@ import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats; @@ -499,7 +500,7 @@ public void testTopicStats() throws Exception { // subscription stats assertEquals(stats.getSubscriptions().keySet().size(), 1); - assertEquals(subStats.consumers.size(), 1); + assertEquals(subStats.getConsumers().size(), 1); Producer producer = pulsarClient.newProducer().topic(topicName).create(); Thread.sleep(timeWaitToSync); @@ -515,14 +516,14 @@ public void testTopicStats() throws Exception { stats = topicRef.getStats(false, false); subStats = stats.getSubscriptions().values().iterator().next(); - assertTrue(subStats.msgRateOut > 0); - assertEquals(subStats.consumers.size(), 1); - assertTrue(subStats.msgThroughputOut > 0); + assertTrue(subStats.getMsgRateOut() > 0); + assertEquals(subStats.getConsumers().size(), 1); + assertTrue(subStats.getMsgThroughputOut() > 0); // consumer stats - assertTrue(subStats.consumers.get(0).msgRateOut > 0.0); - assertTrue(subStats.consumers.get(0).msgThroughputOut > 0.0); - assertEquals(subStats.msgRateRedeliver, 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgRateOut() > 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgThroughputOut() > 0.0); + assertEquals(subStats.getMsgRateRedeliver(), 0.0); producer.close(); consumer.close(); @@ -584,7 +585,7 @@ public void testReplicator() throws Exception { // subscription stats assertEquals(stats.getSubscriptions().keySet().size(), 2); - assertEquals(subStats.consumers.size(), 1); + assertEquals(subStats.getConsumers().size(), 1); Thread.sleep(timeWaitToSync); @@ -653,14 +654,14 @@ public void testReplicator() throws Exception { stats = topicRef.getStats(false, false); subStats = stats.getSubscriptions().values().iterator().next(); - assertTrue(subStats.msgRateOut > 0); - assertEquals(subStats.consumers.size(), 1); - assertTrue(subStats.msgThroughputOut > 0); + assertTrue(subStats.getMsgRateOut() > 0); + assertEquals(subStats.getConsumers().size(), 1); + assertTrue(subStats.getMsgThroughputOut() > 0); // consumer stats - assertTrue(subStats.consumers.get(0).msgRateOut > 0.0); - assertTrue(subStats.consumers.get(0).msgThroughputOut > 0.0); - assertEquals(subStats.msgRateRedeliver, 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgRateOut() > 0.0); + assertTrue(subStats.getConsumers().get(0).getMsgThroughputOut() > 0.0); + assertEquals(subStats.getMsgRateRedeliver(), 0.0); producer.close(); consumer1.close(); @@ -870,9 +871,9 @@ public void testMsgDropStat() throws Exception { NonPersistentPublisherStats npStats = stats.getPublishers().get(0); NonPersistentSubscriptionStats sub1Stats = stats.getSubscriptions().get("subscriber-1"); NonPersistentSubscriptionStats sub2Stats = stats.getSubscriptions().get("subscriber-2"); - assertTrue(npStats.msgDropRate > 0); - assertTrue(sub1Stats.msgDropRate > 0); - assertTrue(sub2Stats.msgDropRate > 0); + assertTrue(npStats.getMsgDropRate() > 0); + assertTrue(sub1Stats.getMsgDropRate() > 0); + assertTrue(sub2Stats.getMsgDropRate() > 0); producer.close(); consumer.close(); @@ -1004,14 +1005,23 @@ void setupReplicationCluster() throws Exception { admin3 = PulsarAdmin.builder().serviceHttpUrl(url3.toString()).build(); // Provision the global namespace - admin1.clusters().createCluster("r1", new ClusterDataImpl(url1.toString(), null, pulsar1.getSafeBrokerServiceUrl(), - pulsar1.getBrokerServiceUrlTls())); - admin1.clusters().createCluster("r2", new ClusterDataImpl(url2.toString(), null, pulsar2.getSafeBrokerServiceUrl(), - pulsar1.getBrokerServiceUrlTls())); - admin1.clusters().createCluster("r3", new ClusterDataImpl(url3.toString(), null, pulsar3.getSafeBrokerServiceUrl(), - pulsar1.getBrokerServiceUrlTls())); - - admin1.clusters().createCluster("global", new ClusterDataImpl("http://global:8080")); + admin1.clusters().createCluster("r1", ClusterData.builder() + .serviceUrl(url1.toString()) + .brokerServiceUrl(pulsar1.getSafeBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .build()); + admin1.clusters().createCluster("r2", ClusterData.builder() + .serviceUrl(url2.toString()) + .brokerServiceUrl(pulsar2.getSafeBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .build()); + admin1.clusters().createCluster("r3", ClusterData.builder() + .serviceUrl(url3.toString()) + .brokerServiceUrl(pulsar3.getSafeBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .build()); + + admin1.clusters().createCluster("global", ClusterData.builder().serviceUrl("http://global:8080").build()); admin1.tenants().createTenant("pulsar", new TenantInfoImpl( Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r2", "r3"))); admin1.namespaces().createNamespace("pulsar/global/ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java index 6bcfcdfb85341..eae107cb8f880 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java @@ -25,6 +25,7 @@ import java.util.Set; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -39,7 +40,7 @@ public void beforeMethod(Method m) throws Exception { } protected void producerBaseSetup() throws Exception { - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns"); 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 a35475142378c..0b5a830464343 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 @@ -24,6 +24,7 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.ServerSocket; +import java.util.Collections; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -44,7 +45,9 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -109,9 +112,10 @@ protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { } @Test public void testFindBrokerWithListenerName() throws Exception { - admin.clusters().createCluster("localhost", new ClusterDataImpl(pulsar.getWebServiceAddress())); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Sets.newHashSet("localhost")); + admin.clusters().createCluster("localhost", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton("localhost")) + .build(); this.admin.tenants().createTenant("public", tenantInfo); this.admin.namespaces().createNamespace("public/default"); @@ -149,9 +153,10 @@ private void doFindBrokerWithListenerName(boolean useHttp) throws Exception { @Test public void testHttpLookupRedirect() throws Exception { - admin.clusters().createCluster("localhost", new ClusterDataImpl(pulsar.getWebServiceAddress())); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Sets.newHashSet("localhost")); + admin.clusters().createCluster("localhost", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton("localhost")) + .build(); this.admin.tenants().createTenant("public", tenantInfo); this.admin.namespaces().createNamespace("public/default"); ClientConfigurationData conf = new ClientConfigurationData(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerStatTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerStatTest.java index 106a6e9cc7aba..cc83ba2c331ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerStatTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerStatTest.java @@ -357,7 +357,7 @@ public void testBatchMessagesRateOut() throws PulsarClientException, Interrupted Thread.sleep(2000); // Two seconds sleep runTest.set(false); pulsar.getBrokerService().updateRates(); - double actualRate = admin.topics().getStats(topicName).msgRateOut; + double actualRate = admin.topics().getStats(topicName).getMsgRateOut(); assertTrue(actualRate > (produceRate / batchSize)); consumer.unsubscribe(); log.info("-- Exiting {} test --", methodName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 4022ad87bb72f..f66f222b702f2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -3456,8 +3456,8 @@ public void testFailOverConsumerPriority() throws Exception { Consumer consumer2 = consumerBuilder.subscribe(); AtomicInteger consumer1Count = new AtomicInteger(0); - admin.topics().getPartitionedStats(topicName, true).partitions.forEach((p, stats) -> { - String activeConsumerName = stats.subscriptions.entrySet().iterator().next().getValue().activeConsumerName; + admin.topics().getPartitionedStats(topicName, true).getPartitions().forEach((p, stats) -> { + String activeConsumerName = stats.getSubscriptions().entrySet().iterator().next().getValue().getActiveConsumerName(); if (activeConsumerName.equals("aaa")) { consumer1Count.incrementAndGet(); } @@ -3476,9 +3476,9 @@ public void testFailOverConsumerPriority() throws Exception { retryStrategically((test) -> { try { Map subsCount = Maps.newHashMap(); - admin.topics().getPartitionedStats(topicName, true).partitions.forEach((p, stats) -> { - String activeConsumerName = stats.subscriptions.entrySet().iterator().next() - .getValue().activeConsumerName; + admin.topics().getPartitionedStats(topicName, true).getPartitions().forEach((p, stats) -> { + String activeConsumerName = stats.getSubscriptions().entrySet().iterator().next() + .getValue().getActiveConsumerName(); subsCount.compute(activeConsumerName, (k, v) -> v != null ? v + 1 : 1); }); return subsCount.size() == 3 && subsCount.get("bbb1").equals(evenDistributionCount) @@ -3492,8 +3492,8 @@ public void testFailOverConsumerPriority() throws Exception { }, 5, 100); Map subsCount = Maps.newHashMap(); - admin.topics().getPartitionedStats(topicName, true).partitions.forEach((p, stats) -> { - String activeConsumerName = stats.subscriptions.entrySet().iterator().next().getValue().activeConsumerName; + admin.topics().getPartitionedStats(topicName, true).getPartitions().forEach((p, stats) -> { + String activeConsumerName = stats.getSubscriptions().entrySet().iterator().next().getValue().getActiveConsumerName(); subsCount.compute(activeConsumerName, (k, v) -> v != null ? v + 1 : 1); }); assertEquals(subsCount.size(), 3); 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 8a7e067bea246..65b4a716afd65 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 @@ -61,9 +61,17 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr final int messageRate = 100; DispatchRate dispatchRate = null; if (DispatchRateType.messageRate.equals(dispatchRateType)) { - dispatchRate = new DispatchRate(messageRate, -1, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); } else { - dispatchRate = new DispatchRate(-1, messageRate, 360); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(messageRate) + .ratePeriodInSecond(360) + .build(); } admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); @@ -147,7 +155,11 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio final String subName = "my-subscriber-name"; final int messageRate = 10; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(1) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate); final int numProducedMessages = 30; @@ -228,7 +240,11 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT final String subName = "my-subscriber-name-" + subscription; final int byteRate = 1000; - DispatchRate dispatchRate = new DispatchRate(-1, byteRate, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(byteRate) + .ratePeriodInSecond(1) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate); final int numProducedMessages = 30; @@ -308,7 +324,11 @@ public void testRateLimitingMultipleConsumers() throws Exception { final String subName = "my-subscriber-name"; final int messageRate = 5; - DispatchRate dispatchRate = new DispatchRate(messageRate, -1, 360); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate); @@ -502,7 +522,11 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { // (2) Update namespace throttling limit int nsMessageRate = 500; - DispatchRate dispatchRate = new DispatchRate(nsMessageRate, 0, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(nsMessageRate) + .dispatchThrottlingRateInByte(0) + .ratePeriodInSecond(1) + .build(); admin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate); subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -515,7 +539,11 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { Assert.assertEquals(nsMessageRate, subRateLimiter.getDispatchRateOnMsg()); // (3) Disable namespace throttling limit will force to take cluster-config - dispatchRate = new DispatchRate(0, 0, 1); + dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(0) + .dispatchThrottlingRateInByte(0) + .ratePeriodInSecond(1) + .build(); admin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate); for (int i = 0; i < 5; i++) { if (subRateLimiter.getDispatchRateOnMsg() == nsMessageRate) { @@ -555,7 +583,11 @@ public void testClosingRateLimiter(SubscriptionType subscription) throws Excepti final String topicName = "persistent://" + namespace + "/closingSubRateLimiter" + subscription.name(); final String subName = "mySubscription" + subscription.name(); - DispatchRate dispatchRate = new DispatchRate(10, 1024, 1); + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(1024) + .ratePeriodInSecond(1) + .build(); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TenantTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TenantTest.java index 79a5cd266fe90..ec5364427cc5d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TenantTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TenantTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Sets; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -47,7 +48,7 @@ protected void cleanup() throws Exception { public void testMaxTenant() throws Exception { conf.setMaxTenants(2); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant("testTenant1", tenantInfo); admin.tenants().createTenant("testTenant2", tenantInfo); @@ -61,7 +62,7 @@ public void testMaxTenant() throws Exception { super.internalCleanup(); conf.setMaxTenants(0); super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); for (int i = 0; i < 10; i++) { admin.tenants().createTenant("testTenant-unlimited" + i, tenantInfo); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerBase.java index 28870a70e8ac4..9923a08b90099 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerBase.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.annotations.*; @@ -102,8 +103,13 @@ protected void internalSetUpForNamespace() throws Exception { admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrlTls.toString()) .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(false) .authentication(AuthenticationTls.class.getName(), authParams).build()); - admin.clusters().createCluster(clusterName, new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster(clusterName, + ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java index 7c57768f145c5..40468411fdfcb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java @@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.slf4j.Logger; @@ -129,7 +130,7 @@ public void testTokenProducerAndConsumer() throws Exception { clientSetup(); // test rest by admin - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java index 28fa7ddfdda95..1696c3b312cf2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.auth.oauth2.AuthenticationFactoryOAuth2; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; @@ -151,7 +152,7 @@ public void testTokenProducerAndConsumer() throws Exception { clientSetup(); // test rest by admin - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); @@ -168,7 +169,7 @@ public void testOAuth2TokenRefreshedWithoutReconnect() throws Exception { clientSetup(); // test rest by admin - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java index 723249e73a18e..8b533b5b450c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java @@ -315,16 +315,16 @@ public void testTopicStats() throws Exception { Reader reader2 = pulsarClient.newReader().topic(topicName).startMessageId(MessageId.earliest).create(); TopicStats stats = admin.topics().getStats(topicName); - assertEquals(stats.subscriptions.size(), 2); + assertEquals(stats.getSubscriptions().size(), 2); reader1.close(); stats = admin.topics().getStats(topicName); - assertEquals(stats.subscriptions.size(), 1); + assertEquals(stats.getSubscriptions().size(), 1); reader2.close(); stats = admin.topics().getStats(topicName); - assertEquals(stats.subscriptions.size(), 0); + assertEquals(stats.getSubscriptions().size(), 0); } @Test @@ -337,16 +337,16 @@ public void testMultiTopicStats() throws Exception { Reader reader2 = pulsarClient.newReader().topic(topicName).startMessageId(MessageId.earliest).create(); TopicStats stats = admin.topics().getPartitionedStats(topicName,true); - assertEquals(stats.subscriptions.size(), 2); + assertEquals(stats.getSubscriptions().size(), 2); reader1.close(); stats = admin.topics().getPartitionedStats(topicName, true); - assertEquals(stats.subscriptions.size(), 1); + assertEquals(stats.getSubscriptions().size(), 1); reader2.close(); stats = admin.topics().getPartitionedStats(topicName, true); - assertEquals(stats.subscriptions.size(), 0); + assertEquals(stats.getSubscriptions().size(), 0); } @Test(dataProvider = "variationsForResetOnLatestMsg") diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerBase.java index 67c1ee2a58b09..467a902a2e2b3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerBase.java @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -38,7 +39,7 @@ public void beforeMethod(Method m) throws Exception { } public void producerBaseSetup() throws Exception { - admin.clusters().createCluster("use", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/use/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java index 6985c09b9f864..a83b4af9fd4f7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java @@ -46,6 +46,7 @@ import org.apache.pulsar.client.admin.internal.JacksonConfigurator; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.tls.NoopHostnameVerifier; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -177,7 +178,7 @@ PulsarAdmin buildAdminClient() throws Exception { @Test public void testSuperUserCanListTenants() throws Exception { try (PulsarAdmin admin = buildAdminClient()) { - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("foobar"), ImmutableSet.of("test"))); @@ -188,7 +189,7 @@ public void testSuperUserCanListTenants() throws Exception { @Test public void testSuperUserCantListNamespaces() throws Exception { try (PulsarAdmin admin = buildAdminClient()) { - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("proxy"), ImmutableSet.of("test"))); @@ -200,7 +201,7 @@ public void testSuperUserCantListNamespaces() throws Exception { @Test public void testAuthorizedUserAsOriginalPrincipal() throws Exception { try (PulsarAdmin admin = buildAdminClient()) { - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("proxy", "user1"), ImmutableSet.of("test"))); @@ -219,7 +220,7 @@ public void testPersistentList() throws Exception { log.info("-- Starting {} test --", methodName); try (PulsarAdmin admin = buildAdminClient()) { - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("foobar"), ImmutableSet.of("test"))); 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 859a8f3b0a858..d90ecb0344524 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 @@ -92,6 +92,7 @@ import org.apache.pulsar.client.impl.schema.writer.JacksonJsonWriter; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.protocol.PulsarHandler; @@ -745,7 +746,7 @@ public long getTimestamp() { public void testCleanProducer() throws Exception { log.info("-- Starting {} test --", methodName); - admin.clusters().createCluster("global", new ClusterDataImpl()); + admin.clusters().createCluster("global", ClusterData.builder().build()); admin.namespaces().createNamespace("my-property/global/lookup"); final int operationTimeOut = 500; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerConfigurationTest.java index 0b55ec4a4344e..553db29f89f91 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerConfigurationTest.java @@ -23,6 +23,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.annotations.AfterMethod; @@ -39,7 +40,7 @@ public class ConsumerConfigurationTest extends MockedPulsarServiceBaseTest { public void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("use", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/use/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/DispatchAccordingPermitsTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/DispatchAccordingPermitsTests.java index fed4e40bd4793..19d43d3bbbadc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/DispatchAccordingPermitsTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/DispatchAccordingPermitsTests.java @@ -100,6 +100,6 @@ public void testFlowPermitsWithMultiBatchesDispatch() throws PulsarAdminExceptio Assert.assertEquals(consumerImpl.incomingMessages.size(), 0); TopicStats stats = admin.topics().getStats(topic); - Assert.assertTrue(stats.subscriptions.get(subName).consumers.get(0).availablePermits > 0); + Assert.assertTrue(stats.getSubscriptions().get(subName).getConsumers().get(0).getAvailablePermits() > 0); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java index 35811dce80177..fcc4a51a936e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java @@ -39,6 +39,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -155,8 +156,12 @@ protected void internalSetUpForNamespace() throws Exception { .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW) .allowTlsInsecureConnection(false) .authentication(AuthenticationKeyStoreTls.class.getName(), authParams).build()); - admin.clusters().createCluster(clusterName, new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster(clusterName, ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuthTest.java index f09f3de02d9b2..e6e4717adf6ac 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuthTest.java @@ -37,6 +37,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -141,8 +142,12 @@ protected void internalSetUpForNamespace() throws Exception { .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW) .allowTlsInsecureConnection(true) .authentication(AuthenticationKeyStoreTls.class.getName(), authParams).build()); - admin.clusters().createCluster(clusterName, new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster(clusterName, ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java index ba279dbd8d590..d4eab77a7c21f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java @@ -139,7 +139,7 @@ public void testLargeMessage(boolean ackReceiptEnabled) throws Exception { PublisherStats producerStats = topic.getStats(false, false).publishers.get(0); - assertTrue(producerStats.chunkedMessageRate > 0); + assertTrue(producerStats.getChunkedMessageRate() > 0); ManagedCursorImpl mcursor = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next(); PositionImpl readPosition = (PositionImpl) mcursor.getReadPosition(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java index 20897cde43c65..771a09e3f6b72 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.raw.MessageParser; import org.apache.pulsar.common.api.raw.RawMessage; @@ -56,7 +57,7 @@ public class MessageParserTest extends MockedPulsarServiceBaseTest { public void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-tenant", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-tenant/my-ns", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MultiTopicsReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MultiTopicsReaderTest.java index 89a974046bea0..31a426e9f9dc8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MultiTopicsReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MultiTopicsReaderTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; @@ -71,7 +72,7 @@ protected void setup() throws Exception { super.internalSetup(); admin.clusters().createCluster("test", - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); Policies policies = new Policies(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java index 9e806b8f0984e..de9eb80a26c14 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java @@ -40,6 +40,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.api.RawReader; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.ClusterDataImpl; @@ -61,7 +62,7 @@ public void setup() throws Exception { super.internalSetup(); admin.clusters().createCluster("test", - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index 55ff8f5174d0c..a04f73b167574 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -71,7 +72,7 @@ protected void setup() throws Exception { super.internalSetup(); admin.clusters().createCluster("test", - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); @@ -332,17 +333,17 @@ public void testRemoveSubscriptionForReaderNeedRemoveCursor() throws IOException .startMessageId(MessageId.earliest) .create(); - Assert.assertEquals(admin.topics().getStats(topic).subscriptions.size(), 2); + Assert.assertEquals(admin.topics().getStats(topic).getSubscriptions().size(), 2); Assert.assertEquals(admin.topics().getInternalStats(topic, false).cursors.size(), 2); reader1.close(); - Assert.assertEquals(admin.topics().getStats(topic).subscriptions.size(), 1); + Assert.assertEquals(admin.topics().getStats(topic).getSubscriptions().size(), 1); Assert.assertEquals(admin.topics().getInternalStats(topic, false).cursors.size(), 1); reader2.close(); - Assert.assertEquals(admin.topics().getStats(topic).subscriptions.size(), 0); + Assert.assertEquals(admin.topics().getStats(topic).getSubscriptions().size(), 0); Assert.assertEquals(admin.topics().getInternalStats(topic, false).cursors.size(), 0); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SchemaDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SchemaDeleteTest.java index 939c6d7a0ce35..9b472c94c829e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SchemaDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SchemaDeleteTest.java @@ -26,6 +26,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -45,7 +46,7 @@ protected void setup() throws Exception { this.conf.setBrokerDeleteInactiveTopicsFrequencySeconds(5); admin.clusters().createCluster("test", - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index 8ad2c0cd77334..cc153b249de70 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.TopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.SubscriptionStats; @@ -917,12 +918,12 @@ public int choosePartition(Message msg, TopicMetadata metadata) { Map activeConsumers = new HashMap<>(); PartitionedTopicStats stats = admin.topics().getPartitionedStats(topicName, true); - for (TopicStats value : stats.partitions.values()) { - for (SubscriptionStats subscriptionStats : value.subscriptions.values()) { - assertTrue(subscriptionStats.activeConsumerName.equals(consumer_1.getConsumerName()) - || subscriptionStats.activeConsumerName.equals(consumer_2.getConsumerName())); - activeConsumers.putIfAbsent(subscriptionStats.activeConsumerName, new AtomicInteger(0)); - activeConsumers.get(subscriptionStats.activeConsumerName).incrementAndGet(); + for (TopicStats value : stats.getPartitions().values()) { + for (SubscriptionStats subscriptionStats : value.getSubscriptions().values()) { + assertTrue(subscriptionStats.getActiveConsumerName().equals(consumer_1.getConsumerName()) + || subscriptionStats.getActiveConsumerName().equals(consumer_2.getConsumerName())); + activeConsumers.putIfAbsent(subscriptionStats.getActiveConsumerName(), new AtomicInteger(0)); + activeConsumers.get(subscriptionStats.getActiveConsumerName()).incrementAndGet(); } } assertEquals(activeConsumers.get(consumer_1.getConsumerName()).get(), 2); @@ -959,7 +960,7 @@ public void testDefaultBacklogTTL() throws Exception { final String topicName = "persistent://" + namespace + "/expiry"; final String subName = "expiredSub"; - admin.clusters().createCluster("use", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("prop", new TenantInfoImpl(null, Sets.newHashSet("use"))); admin.namespaces().createNamespace(namespace); @@ -1093,7 +1094,7 @@ public void multiTopicsInDifferentNameSpace() throws PulsarAdminException, Pulsa topics.add("persistent://prop/use/ns-abc/topic-1"); topics.add("persistent://prop/use/ns-abc/topic-2"); topics.add("persistent://prop/use/ns-abc1/topic-3"); - admin.clusters().createCluster("use", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("prop", new TenantInfoImpl(null, Sets.newHashSet("use"))); admin.namespaces().createNamespace("prop/use/ns-abc"); admin.namespaces().createNamespace("prop/use/ns-abc1"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index f6bcbeea65282..960beaac12d22 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -65,6 +65,7 @@ import org.apache.pulsar.common.api.proto.CommandAck; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -99,7 +100,7 @@ protected void setup() throws Exception { String[] brokerServiceUrlArr = getPulsarServiceList().get(0).getBrokerServiceUrl().split(":"); String webServicePort = brokerServiceUrlArr[brokerServiceUrlArr.length -1]; - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl("http://localhost:" + webServicePort)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl("http://localhost:" + webServicePort).build()); admin.tenants().createTenant(TENANT, new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet(CLUSTER_NAME))); admin.namespaces().createNamespace(NAMESPACE1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java index ff03e59b7c1b7..2d3411b7b8d3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java @@ -52,6 +52,7 @@ import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.impl.RawMessageImpl; import org.apache.pulsar.common.api.proto.MessageIdData; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.FutureUtil; @@ -79,7 +80,7 @@ public void setup() throws Exception { super.internalSetup(); admin.clusters().createCluster("use", - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/use/my-ns"); 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 2cb6d9d6aa109..c9e0d95f7ffdc 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 @@ -62,6 +62,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.RetentionPolicies; @@ -83,7 +84,7 @@ public class CompactionTest extends MockedPulsarServiceBaseTest { public void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("use", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/use/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 438a1bf96c090..57a2146c8efec 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.impl.RawMessageImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -63,7 +64,7 @@ public void setup() throws Exception { super.internalSetup(); admin.clusters().createCluster("use", - new ClusterDataImpl(pulsar.getWebServiceAddress())); + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); admin.namespaces().createNamespace("my-property/use/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java index ad5ac49999deb..217dc63e6380c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java @@ -109,7 +109,8 @@ public void testRedirectUrlWithServerStarted() throws Exception { assertEquals(hitBrokerService(HttpMethod.POST, postRequestUrl, Lists.newArrayList("use")), "Need to authenticate to perform the request"); - assertEquals(hitBrokerService(HttpMethod.PUT, putRequestUrl, new BundlesData(1)), "Need to authenticate to perform the request"); + assertEquals(hitBrokerService(HttpMethod.PUT, putRequestUrl, + BundlesData.builder().numBundles(1).build()), "Need to authenticate to perform the request"); assertEquals(hitBrokerService(HttpMethod.GET, getRequestUrl, null), "Need to authenticate to perform the request"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java index b055d1b70f8be..e7173a23e39b5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java @@ -60,8 +60,10 @@ import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; @@ -171,7 +173,7 @@ void setup(Method method) throws Exception { primaryHost = pulsar.getWebServiceAddress(); // update cluster metadata - ClusterDataImpl clusterData = new ClusterDataImpl(brokerWebServiceUrl.toString()); + ClusterData clusterData = ClusterData.builder().serviceUrl(brokerWebServiceUrl.toString()).build(); superUserAdmin.clusters().updateCluster(config.getClusterName(), clusterData); ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(this.workerConfig.getPulsarServiceUrl()) @@ -186,9 +188,10 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { } pulsarClient = clientBuilder.build(); - TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.getAdminRoles().add(ADMIN_SUBJECT); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + TenantInfo propAdmin = TenantInfo.builder() + .adminRoles(Collections.singleton(ADMIN_SUBJECT)) + .allowedClusters(Collections.singleton("use")) + .build(); superUserAdmin.tenants().updateTenant(TENANT, propAdmin); @@ -198,8 +201,9 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { superUserAdmin.namespaces().setNamespaceReplicationClusters(replNamespace, clusters); // create another test tenant and namespace - propAdmin = new TenantInfoImpl(); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + propAdmin = TenantInfo.builder() + .allowedClusters(Collections.singleton("use")) + .build(); superUserAdmin.tenants().createTenant(TENANT2, propAdmin); superUserAdmin.namespaces().createNamespace( TENANT2 + "/" + NAMESPACE); @@ -345,13 +349,13 @@ public void testAuthorizationWithAnonymousUser() throws Exception { assertTrue(retryStrategically((test) -> { try { return admin1.functions().getFunctionStatus(TENANT, NAMESPACE, functionName).getNumRunning() == 1 - && admin1.topics().getStats(sourceTopic).subscriptions.size() == 1; + && admin1.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150)); // validate pulsar sink consumer has started on the topic - assertEquals(admin1.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin1.topics().getStats(sourceTopic).getSubscriptions().size(), 1); // create a producer that creates a topic at broker try (Producer producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create(); @@ -364,8 +368,8 @@ public void testAuthorizationWithAnonymousUser() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin1.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0; + SubscriptionStats subStats = admin1.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0; } catch (PulsarAdminException e) { return false; } @@ -379,7 +383,7 @@ public void testAuthorizationWithAnonymousUser() throws Exception { // validate pulsar-sink consumer has consumed all messages and delivered to Pulsar sink but unacked // messages // due to publish failure - assertNotEquals(admin1.topics().getStats(sourceTopic).subscriptions.values().iterator().next().unackedMessages, + assertNotEquals(admin1.topics().getStats(sourceTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), totalMsgs); // test update functions @@ -538,7 +542,7 @@ public void testAuthorizationWithAnonymousUser() throws Exception { assertTrue(retryStrategically((test) -> { try { TopicStats stats = admin1.topics().getStats(sourceTopic); - boolean done = stats.subscriptions.size() == 0; + boolean done = stats.getSubscriptions().size() == 0; if (!done) { log.info("Topic subscription is not cleaned up yet : {}", stats); } @@ -621,13 +625,13 @@ public void testAuthorization() throws Exception { assertTrue(retryStrategically((test) -> { try { return admin1.functions().getFunctionStatus(TENANT, NAMESPACE, functionName).getNumRunning() == 1 - && admin1.topics().getStats(sourceTopic).subscriptions.size() == 1; + && admin1.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150)); // validate pulsar sink consumer has started on the topic - assertEquals(admin1.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin1.topics().getStats(sourceTopic).getSubscriptions().size(), 1); // create a producer that creates a topic at broker try(Producer producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create(); @@ -640,8 +644,8 @@ public void testAuthorization() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin1.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0; + SubscriptionStats subStats = admin1.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0; } catch (PulsarAdminException e) { return false; } @@ -655,7 +659,7 @@ public void testAuthorization() throws Exception { // validate pulsar-sink consumer has consumed all messages and delivered to Pulsar sink but unacked // messages // due to publish failure - assertNotEquals(admin1.topics().getStats(sourceTopic).subscriptions.values().iterator().next().unackedMessages, + assertNotEquals(admin1.topics().getStats(sourceTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), totalMsgs); } @@ -815,7 +819,7 @@ public void testAuthorization() throws Exception { assertTrue(retryStrategically((test) -> { try { TopicStats stats = admin1.topics().getStats(sourceTopic); - boolean done = stats.subscriptions.size() == 0; + boolean done = stats.getSubscriptions().size() == 0; if (!done) { log.info("Topic subscription is not cleaned up yet : {}", stats); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java index 1641022a236d9..9477af4f7846c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java @@ -73,10 +73,12 @@ import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.PublisherStats; import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; @@ -250,7 +252,7 @@ void setup(Method method) throws Exception { primaryHost = pulsar.getWebServiceAddress(); // create cluster metadata - ClusterDataImpl clusterData = new ClusterDataImpl(urlTls.toString()); + ClusterData clusterData = ClusterData.builder().serviceUrl(urlTls.toString()).build(); admin.clusters().createCluster(config.getClusterName(), clusterData); ClientBuilder clientBuilder = PulsarClient.builder() @@ -268,9 +270,10 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { } pulsarClient = clientBuilder.build(); - TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.getAdminRoles().add("superUser"); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList(CLUSTER))); + TenantInfo propAdmin = TenantInfo.builder() + .adminRoles(Collections.singleton("superUser")) + .allowedClusters(Sets.newHashSet(Lists.newArrayList(CLUSTER))) + .build(); admin.tenants().createTenant(tenant, propAdmin); // setting up simple web sever to test submitting function via URL @@ -439,13 +442,13 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl, int parallelis boolean result = false; TopicStats topicStats = admin.topics().getStats(sourceTopic); - if (topicStats.subscriptions.containsKey(subscriptionName) - && topicStats.subscriptions.get(subscriptionName).consumers.size() == parallelism) { - for (ConsumerStats consumerStats : topicStats.subscriptions.get(subscriptionName).consumers) { - result = consumerStats.availablePermits == 1000 - && consumerStats.metadata != null - && consumerStats.metadata.containsKey("id") - && consumerStats.metadata.get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, functionName)); + if (topicStats.getSubscriptions().containsKey(subscriptionName) + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().size() == parallelism) { + for (ConsumerStats consumerStats : topicStats.getSubscriptions().get(subscriptionName).getConsumers()) { + result = consumerStats.getAvailablePermits() == 1000 + && consumerStats.getMetadata() != null + && consumerStats.getMetadata().containsKey("id") + && consumerStats.getMetadata().get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, functionName)); } } return result; @@ -455,8 +458,8 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl, int parallelis }, 50, 150)); // validate pulsar sink consumer has started on the topic TopicStats stats = admin.topics().getStats(sourceTopic); - assertTrue(stats.subscriptions.get(subscriptionName) != null - && !stats.subscriptions.get(subscriptionName).consumers.isEmpty()); + assertTrue(stats.getSubscriptions().get(subscriptionName) != null + && !stats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty()); int totalMsgs = 5; for (int i = 0; i < totalMsgs; i++) { @@ -465,8 +468,8 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl, int parallelis } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0; } catch (PulsarAdminException e) { return false; } @@ -481,7 +484,7 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl, int parallelis // validate pulsar-sink consumer has consumed all messages and delivered to Pulsar sink but unacked messages // due to publish failure - assertNotEquals(admin.topics().getStats(sourceTopic).subscriptions.values().iterator().next().unackedMessages, + assertNotEquals(admin.topics().getStats(sourceTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), totalMsgs); // validate prometheus metrics @@ -520,20 +523,20 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl, int parallelis retryStrategically((test) -> { try { TopicStats topicStats = admin.topics().getStats(sourceTopic); - return topicStats.subscriptions.get(subscriptionName) != null - && topicStats.subscriptions.get(subscriptionName).consumers.isEmpty(); + return topicStats.getSubscriptions().get(subscriptionName) != null + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty(); } catch (PulsarAdminException e) { return false; } }, 20, 150); TopicStats topicStats = admin.topics().getStats(sourceTopic); - assertTrue(topicStats.subscriptions.get(subscriptionName) != null - && topicStats.subscriptions.get(subscriptionName).consumers.isEmpty()); + assertTrue(topicStats.getSubscriptions().get(subscriptionName) != null + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty()); retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic).publishers.size() == 0); + return (admin.topics().getStats(sinkTopic).getPublishers().size() == 0); } catch (PulsarAdminException e) { if (e.getStatusCode() == 404) { return true; @@ -543,7 +546,7 @@ private void testE2EPulsarFunctionLocalRun(String jarFilePathUrl, int parallelis }, 10, 150); try { - assertEquals(admin.topics().getStats(sinkTopic).publishers.size(), 0); + assertEquals(admin.topics().getStats(sinkTopic).getPublishers().size(), 0); } catch (PulsarAdminException e) { if (e.getStatusCode() != 404) { fail(); @@ -615,8 +618,8 @@ private void testAvroFunctionLocalRun(String jarFilePathUrl) throws Exception { retryStrategically((test) -> { try { TopicStats stats = admin.topics().getStats(sourceTopic); - return stats.subscriptions.get(subscriptionName) != null - && !stats.subscriptions.get(subscriptionName).consumers.isEmpty(); + return stats.getSubscriptions().get(subscriptionName) != null + && !stats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty(); } catch (PulsarAdminException e) { return false; } @@ -641,14 +644,14 @@ private void testAvroFunctionLocalRun(String jarFilePathUrl) throws Exception { } // validate pulsar-sink consumer has consumed all messages - assertNotEquals(admin.topics().getStats(sinkTopic).subscriptions.values().iterator().next().unackedMessages, 0); + assertNotEquals(admin.topics().getStats(sinkTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), 0); localRunner.stop(); retryStrategically((test) -> { try { TopicStats topicStats = admin.topics().getStats(sourceTopic); - return topicStats.subscriptions.get(subscriptionName) != null - && topicStats.subscriptions.get(subscriptionName).consumers.isEmpty(); + return topicStats.getSubscriptions().get(subscriptionName) != null + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty(); } catch (PulsarAdminException e) { return false; } @@ -737,7 +740,7 @@ private void testPulsarSourceLocalRun(String jarFilePathUrl, int parallelism) th Assert.assertTrue(retryStrategically((test) -> { try { - return admin.topics().getStats(sinkTopic).publishers.size() == parallelism; + return admin.topics().getStats(sinkTopic).getPublishers().size() == parallelism; } catch (PulsarAdminException e) { return false; } @@ -747,11 +750,11 @@ private void testPulsarSourceLocalRun(String jarFilePathUrl, int parallelism) th try { boolean result = false; TopicStats sourceStats = admin.topics().getStats(sinkTopic); - if (sourceStats.publishers.size() == parallelism) { - for (PublisherStats publisher : sourceStats.publishers) { - result = publisher.metadata != null - && publisher.metadata.containsKey("id") - && publisher.metadata.get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); + if (sourceStats.getPublishers().size() == parallelism) { + for (PublisherStats publisher : sourceStats.getPublishers()) { + result = publisher.getMetadata() != null + && publisher.getMetadata().containsKey("id") + && publisher.getMetadata().get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); } } @@ -763,13 +766,13 @@ private void testPulsarSourceLocalRun(String jarFilePathUrl, int parallelism) th Assert.assertTrue(retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic).publishers.size() == parallelism) + return (admin.topics().getStats(sinkTopic).getPublishers().size() == parallelism) && (admin.topics().getInternalStats(sinkTopic, false).numberOfEntries > 4); } catch (PulsarAdminException e) { return false; } }, 50, 150)); - assertEquals(admin.topics().getStats(sinkTopic).publishers.size(), parallelism); + assertEquals(admin.topics().getStats(sinkTopic).getPublishers().size(), parallelism); // validate prometheus metrics String prometheusMetrics = PulsarFunctionTestUtils.getPrometheusMetrics(metricsPort); @@ -803,14 +806,14 @@ private void testPulsarSourceLocalRun(String jarFilePathUrl, int parallelism) th Assert.assertTrue(retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic).publishers.size() == 0); + return (admin.topics().getStats(sinkTopic).getPublishers().size() == 0); } catch (PulsarAdminException e) { return e.getStatusCode() == 404; } }, 10, 150)); try { - assertEquals(admin.topics().getStats(sinkTopic).publishers.size(), 0); + assertEquals(admin.topics().getStats(sinkTopic).getPublishers().size(), 0); } catch (PulsarAdminException e) { if (e.getStatusCode() != 404) { fail(); @@ -893,10 +896,10 @@ private void testPulsarSinkLocalRun(String jarFilePathUrl, int parallelism) thro try { boolean result = false; TopicStats topicStats = admin.topics().getStats(sourceTopic); - if (topicStats.subscriptions.containsKey(subscriptionName) - && topicStats.subscriptions.get(subscriptionName).consumers.size() == parallelism) { - for (ConsumerStats consumerStats : topicStats.subscriptions.get(subscriptionName).consumers) { - result = consumerStats.availablePermits == 1000; + if (topicStats.getSubscriptions().containsKey(subscriptionName) + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().size() == parallelism) { + for (ConsumerStats consumerStats : topicStats.getSubscriptions().get(subscriptionName).getConsumers()) { + result = consumerStats.getAvailablePermits() == 1000; } } return result; @@ -912,8 +915,8 @@ private void testPulsarSinkLocalRun(String jarFilePathUrl, int parallelism) thro } Assert.assertTrue(retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -955,16 +958,16 @@ private void testPulsarSinkLocalRun(String jarFilePathUrl, int parallelism) thro Assert.assertTrue(retryStrategically((test) -> { try { TopicStats stats = admin.topics().getStats(sourceTopic); - return stats.subscriptions.get(subscriptionName) != null - && stats.subscriptions.get(subscriptionName).consumers.isEmpty(); + return stats.getSubscriptions().get(subscriptionName) != null + && stats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty(); } catch (PulsarAdminException e) { return false; } }, 20, 150)); TopicStats topicStats = admin.topics().getStats(sourceTopic); - assertTrue(topicStats.subscriptions.get(subscriptionName) != null - && topicStats.subscriptions.get(subscriptionName).consumers.isEmpty()); + assertTrue(topicStats.getSubscriptions().get(subscriptionName) != null + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().isEmpty()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java index 15805f9beb75c..3b704b6436f4c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java @@ -63,9 +63,11 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.FunctionStatsImpl; import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -176,7 +178,7 @@ void setup(Method method) throws Exception { primaryHost = pulsar.getWebServiceAddress(); // update cluster metadata - ClusterDataImpl clusterData = new ClusterDataImpl(urlTls.toString()); + ClusterData clusterData = ClusterData.builder().serviceUrl(urlTls.toString()).build(); admin.clusters().updateCluster(config.getClusterName(), clusterData); ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(this.workerConfig.getPulsarServiceUrl()); @@ -192,9 +194,10 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { } pulsarClient = clientBuilder.build(); - TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.getAdminRoles().add("superUser"); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + TenantInfo propAdmin = TenantInfo.builder() + .adminRoles(Collections.singleton("superUser")) + .allowedClusters(Collections.singleton("use")) + .build(); admin.tenants().updateTenant(tenant, propAdmin); System.setProperty(JAVA_INSTANCE_JAR_PROPERTY, @@ -309,13 +312,13 @@ public void testPulsarFunctionState() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate pulsar sink consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); int totalMsgs = 5; for (int i = 0; i < totalMsgs; i++) { @@ -324,8 +327,8 @@ public void testPulsarFunctionState() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0; } catch (PulsarAdminException e) { return false; } @@ -351,7 +354,7 @@ public void testPulsarFunctionState() throws Exception { // validate pulsar-sink consumer has consumed all messages and delivered to Pulsar sink but unacked messages // due to publish failure - assertNotEquals(admin.topics().getStats(sourceTopic).subscriptions.values().iterator().next().unackedMessages, + assertNotEquals(admin.topics().getStats(sourceTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), totalMsgs); // delete functions @@ -359,14 +362,14 @@ public void testPulsarFunctionState() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); tempDirectory.assertThatFunctionDownloadTempFilesHaveBeenDeleted(); } @@ -404,16 +407,16 @@ public void testMultipleAddress() throws Exception { pulsarAdmin.functions().createFunction(functionConfig, jarFile.getAbsolutePath()); retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); admin.functions().deleteFunction(tenant, namespacePortion, functionName); retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } @@ -447,13 +450,13 @@ public void testPulsarFunctionBKCleanup() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate pulsar sink consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); int totalMsgs = 5; for (int i = 0; i < totalMsgs; i++) { @@ -462,8 +465,8 @@ public void testPulsarFunctionBKCleanup() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0; } catch (PulsarAdminException e) { return false; } @@ -489,7 +492,7 @@ public void testPulsarFunctionBKCleanup() throws Exception { // validate pulsar-sink consumer has consumed all messages and delivered to Pulsar sink but unacked messages // due to publish failure - assertNotEquals(admin.topics().getStats(sourceTopic).subscriptions.values().iterator().next().unackedMessages, + assertNotEquals(admin.topics().getStats(sourceTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), totalMsgs); // delete functions @@ -497,14 +500,14 @@ public void testPulsarFunctionBKCleanup() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); tempDirectory.assertThatFunctionDownloadTempFilesHaveBeenDeleted(); 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 23ae6ef2eb6a6..1a22358004ed0 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 @@ -26,6 +26,7 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -40,6 +41,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.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -165,8 +167,9 @@ void setup() throws Exception { leaderAdmin = pulsarAdmins[0]; Thread.sleep(1000); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Sets.newHashSet(testCluster)); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton(testCluster)) + .build(); pulsarAdmins[0].tenants().createTenant(testTenant, tenantInfo); pulsarAdmins[0].namespaces().createNamespace(testNamespace, 16); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java index 76e5b64d76f7e..019ca0bd577f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java @@ -28,6 +28,7 @@ import com.google.gson.Gson; import java.lang.reflect.Method; import java.net.URI; +import java.util.Collections; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -44,7 +45,9 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.proto.Function.Assignment; @@ -108,7 +111,7 @@ void setup(Method method) throws Exception { primaryHost = pulsar.getWebServiceAddress(); // update cluster metadata - final ClusterDataImpl clusterData = new ClusterDataImpl(pulsar.getBrokerServiceUrl()); + final ClusterData clusterData = ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); admin.clusters().updateCluster(config.getClusterName(), clusterData); final ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(this.workerConfig.getPulsarServiceUrl()); @@ -117,9 +120,10 @@ void setup(Method method) throws Exception { } pulsarClient = clientBuilder.build(); - final TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.getAdminRoles().add("superUser"); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + TenantInfo propAdmin = TenantInfo.builder() + .adminRoles(Collections.singleton("superUser")) + .allowedClusters(Collections.singleton("use")) + .build(); admin.tenants().updateTenant(tenant, propAdmin); Thread.sleep(100); @@ -196,16 +200,16 @@ public void testFunctionAssignments() throws Exception { admin.functions().createFunctionWithUrl(functionConfig, jarFilePathUrl); retryStrategically((test) -> { try { - return admin.topics().getStats(sinkTopic).subscriptions.size() == 1 - && admin.topics().getStats(sinkTopic).subscriptions.values().iterator().next().consumers + return admin.topics().getStats(sinkTopic).getSubscriptions().size() == 1 + && admin.topics().getStats(sinkTopic).getSubscriptions().values().iterator().next().getConsumers() .size() == 2; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate 2 instances have been started - assertEquals(admin.topics().getStats(sinkTopic).subscriptions.size(), 1); - assertEquals(admin.topics().getStats(sinkTopic).subscriptions.values().iterator().next().consumers.size(), 2); + assertEquals(admin.topics().getStats(sinkTopic).getSubscriptions().size(), 1); + assertEquals(admin.topics().getStats(sinkTopic).getSubscriptions().values().iterator().next().getConsumers().size(), 2); // (2) Update function with 1 instance functionConfig.setParallelism(1); @@ -213,8 +217,8 @@ public void testFunctionAssignments() throws Exception { admin.functions().updateFunctionWithUrl(functionConfig, jarFilePathUrl); retryStrategically((test) -> { try { - return admin.topics().getStats(sinkTopic).subscriptions.size() == 1 - && admin.topics().getStats(sinkTopic).subscriptions.values().iterator().next().consumers + return admin.topics().getStats(sinkTopic).getSubscriptions().size() == 1 + && admin.topics().getStats(sinkTopic).getSubscriptions().values().iterator().next().getConsumers() .size() == 1; } catch (PulsarAdminException e) { return false; @@ -222,7 +226,7 @@ public void testFunctionAssignments() throws Exception { }, 50, 150); // validate pulsar sink consumer has started on the topic log.info("admin.topics().getStats(sinkTopic): {}", new Gson().toJson(admin.topics().getStats(sinkTopic))); - assertEquals(admin.topics().getStats(sinkTopic).subscriptions.values().iterator().next().consumers.size(), 1); + assertEquals(admin.topics().getStats(sinkTopic).getSubscriptions().values().iterator().next().getConsumers().size(), 1); } @Test(timeOut = 60000, enabled = false) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java index a3041977044a7..1df4be1c71d0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java @@ -32,6 +32,7 @@ import java.lang.reflect.Method; import java.nio.file.Files; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -50,7 +51,9 @@ import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -179,7 +182,7 @@ public void setup(Method method) throws Exception { primaryHost = String.format("http://%s:%d", "localhost", pulsar.getListenPortHTTP().get()); // update cluster metadata - ClusterDataImpl clusterData = new ClusterDataImpl(pulsar.getBrokerServiceUrlTls()); + ClusterData clusterData = ClusterData.builder().serviceUrl(pulsar.getBrokerServiceUrlTls()).build(); admin.clusters().updateCluster(config.getClusterName(), clusterData); ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(this.workerConfig.getPulsarServiceUrl()); @@ -195,9 +198,10 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { } pulsarClient = clientBuilder.build(); - TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.getAdminRoles().add("superUser"); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + TenantInfo propAdmin = TenantInfo.builder() + .adminRoles(Collections.singleton("superUser")) + .allowedClusters(Collections.singleton("use")) + .build(); admin.tenants().updateTenant(tenant, propAdmin); assertTrue(getPulsarIODataGeneratorNar().exists(), "pulsar-io-data-generator.nar file " diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarBatchSourceE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarBatchSourceE2ETest.java index 026d98846c3db..d1cd1dc0614e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarBatchSourceE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarBatchSourceE2ETest.java @@ -59,7 +59,7 @@ private void testPulsarBatchSourceStats(String jarFilePathUrl) throws Exception retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic).publishers.size() == 1); + return (admin.topics().getStats(sinkTopic).getPublishers().size() == 1); } catch (PulsarAdminException e) { return false; } @@ -78,29 +78,29 @@ private void testPulsarBatchSourceStats(String jarFilePathUrl) throws Exception retryStrategically((test) -> { try { TopicStats sourceStats = admin.topics().getStats(sinkTopic2); - return sourceStats.publishers.size() == 1 - && sourceStats.publishers.get(0).metadata != null - && sourceStats.publishers.get(0).metadata.containsKey("id") - && sourceStats.publishers.get(0).metadata.get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); + return sourceStats.getPublishers().size() == 1 + && sourceStats.getPublishers().get(0).getMetadata() != null + && sourceStats.getPublishers().get(0).getMetadata().containsKey("id") + && sourceStats.getPublishers().get(0).getMetadata().get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); } catch (PulsarAdminException e) { return false; } }, 50, 150); TopicStats sourceStats = admin.topics().getStats(sinkTopic2); - assertEquals(sourceStats.publishers.size(), 1); - assertNotNull(sourceStats.publishers.get(0).metadata); - assertTrue(sourceStats.publishers.get(0).metadata.containsKey("id")); - assertEquals(sourceStats.publishers.get(0).metadata.get("id"), String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); + assertEquals(sourceStats.getPublishers().size(), 1); + assertNotNull(sourceStats.getPublishers().get(0).getMetadata()); + assertTrue(sourceStats.getPublishers().get(0).getMetadata().containsKey("id")); + assertEquals(sourceStats.getPublishers().get(0).getMetadata().get("id"), String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic2).publishers.size() == 1) && (admin.topics().getInternalStats(sinkTopic2, false).numberOfEntries > 4); + return (admin.topics().getStats(sinkTopic2).getPublishers().size() == 1) && (admin.topics().getInternalStats(sinkTopic2, false).numberOfEntries > 4); } catch (PulsarAdminException e) { return false; } }, 50, 150); - assertEquals(admin.topics().getStats(sinkTopic2).publishers.size(), 1); + assertEquals(admin.topics().getStats(sinkTopic2).getPublishers().size(), 1); String prometheusMetrics = PulsarFunctionTestUtils.getPrometheusMetrics(pulsar.getListenPortHTTP().get()); log.info("prometheusMetrics: {}", prometheusMetrics); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java index a4ddd66807591..ddb1fb3d736d3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java @@ -26,6 +26,7 @@ import java.lang.reflect.Method; import java.net.URL; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -43,7 +44,9 @@ import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; @@ -139,7 +142,7 @@ void setup(Method method) throws Exception { primaryHost = pulsar.getWebServiceAddress(); // update cluster metadata - ClusterDataImpl clusterData = new ClusterDataImpl(urlTls.toString()); + ClusterData clusterData = ClusterData.builder().serviceUrl(urlTls.toString()).build(); admin.clusters().updateCluster(config.getClusterName(), clusterData); ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(this.workerConfig.getPulsarServiceUrl()); @@ -155,8 +158,9 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { } pulsarClient = clientBuilder.build(); - TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + TenantInfo propAdmin = TenantInfo.builder() + .allowedClusters(Collections.singleton("use")) + .build(); admin.tenants().updateTenant(tenant, propAdmin); Thread.sleep(100); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java index ebd034efbc1d3..cdd1dc3ea7ca4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.common.policies.data.FunctionStatsImpl; import org.apache.pulsar.common.policies.data.FunctionStatus; import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.compaction.TwoPhaseCompactor; @@ -129,16 +130,16 @@ private void testE2EPulsarFunction(String jarFilePathUrl) throws Exception { Awaitility.await().ignoreExceptions().untilAsserted(() -> { TopicStats topicStats = admin.topics().getStats(sinkTopic2); - assertEquals(topicStats.publishers.size(), 2); - assertNotNull(topicStats.publishers.get(0).metadata); - assertTrue(topicStats.publishers.get(0).metadata.containsKey("id")); - assertEquals(topicStats.publishers.get(0).metadata.get("id"), + assertEquals(topicStats.getPublishers().size(), 2); + assertNotNull(topicStats.getPublishers().get(0).getMetadata()); + assertTrue(topicStats.getPublishers().get(0).getMetadata().containsKey("id")); + assertEquals(topicStats.getPublishers().get(0).getMetadata().get("id"), String.format("%s/%s/%s", tenant, namespacePortion, functionName)); }); Awaitility.await().ignoreExceptions().untilAsserted(() -> { // validate pulsar sink consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); }); int totalMsgs = 5; @@ -148,8 +149,8 @@ private void testE2EPulsarFunction(String jarFilePathUrl) throws Exception { } Awaitility.await().ignoreExceptions().untilAsserted(() -> { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - assertEquals(subStats.unackedMessages, 0); + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + assertEquals(subStats.getUnackedMessages(), 0); }); Message msg = consumer.receive(5, TimeUnit.SECONDS); @@ -158,7 +159,7 @@ private void testE2EPulsarFunction(String jarFilePathUrl) throws Exception { // validate pulsar-sink consumer has consumed all messages and delivered to Pulsar sink but unacked messages // due to publish failure - assertNotEquals(admin.topics().getStats(sourceTopic).subscriptions.values().iterator().next().unackedMessages, + assertNotEquals(admin.topics().getStats(sourceTopic).getSubscriptions().values().iterator().next().getUnackedMessages(), totalMsgs); // delete functions @@ -166,7 +167,7 @@ private void testE2EPulsarFunction(String jarFilePathUrl) throws Exception { Awaitility.await().ignoreExceptions().untilAsserted(() -> { // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); }); tempDirectory.assertThatFunctionDownloadTempFilesHaveBeenDeleted(); @@ -281,13 +282,13 @@ public void testPulsarFunctionStats() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate pulsar sink consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); // validate stats are empty FunctionRuntimeManager functionRuntimeManager = functionsWorkerService.getFunctionRuntimeManager(); @@ -431,8 +432,8 @@ public void testPulsarFunctionStats() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -577,14 +578,14 @@ public void testPulsarFunctionStats() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); tempDirectory.assertThatFunctionDownloadTempFilesHaveBeenDeleted(); } @@ -617,13 +618,13 @@ public void testPulsarFunctionStatus() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate pulsar sink consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); int totalMsgs = 10; for (int i = 0; i < totalMsgs; i++) { @@ -632,8 +633,8 @@ public void testPulsarFunctionStatus() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -660,14 +661,14 @@ public void testPulsarFunctionStatus() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); } @Test(dataProvider = "validRoleName") @@ -682,9 +683,11 @@ public void testAuthorization(boolean validRoleName) throws Exception { admin.namespaces().setNamespaceReplicationClusters(replNamespace, clusters); String roleName = validRoleName ? "superUser" : "invalid"; - TenantInfoImpl propAdmin = new TenantInfoImpl(); - propAdmin.getAdminRoles().add(roleName); - propAdmin.setAllowedClusters(Sets.newHashSet(Lists.newArrayList("use"))); + + TenantInfo propAdmin = TenantInfo.builder() + .adminRoles(Collections.singleton(roleName)) + .allowedClusters(Collections.singleton("use")) + .build(); admin.tenants().updateTenant(tenant, propAdmin); String jarFilePathUrl = getPulsarApiExamplesJar().toURI().toString(); @@ -736,45 +739,45 @@ public void testFunctionStopAndRestartApi() throws Exception { retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats != null && subStats.consumers.size() == 1; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats != null && subStats.getConsumers().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - assertEquals(subStats.consumers.size(), 1); + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + assertEquals(subStats.getConsumers().size(), 1); // it should stop consumer : so, check none of the consumer connected on subscription admin.functions().stopFunction(tenant, namespacePortion, functionName); retryStrategically((test) -> { try { - SubscriptionStats subStat = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStat != null && subStat.consumers.size() == 0; + SubscriptionStats subStat = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStat != null && subStat.getConsumers().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); - subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - assertEquals(subStats.consumers.size(), 0); + subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + assertEquals(subStats.getConsumers().size(), 0); // it should restart consumer : so, check if consumer came up again after restarting function admin.functions().restartFunction(tenant, namespacePortion, functionName); retryStrategically((test) -> { try { - SubscriptionStats subStat = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStat != null && subStat.consumers.size() == 1; + SubscriptionStats subStat = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStat != null && subStat.getConsumers().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); - subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - assertEquals(subStats.consumers.size(), 1); + subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + assertEquals(subStats.getConsumers().size(), 1); producer.close(); } @@ -820,13 +823,13 @@ public void testFunctionAutomaticSubCleanup() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate pulsar source consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); // test update cleanup subscription functionConfig.setCleanupSubscription(true); @@ -848,9 +851,9 @@ public void testFunctionAutomaticSubCleanup() throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get( + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get( InstanceUtils.getDefaultSubscriptionName(tenant, namespacePortion, functionName)); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -877,14 +880,14 @@ public void testFunctionAutomaticSubCleanup() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); /** test do not cleanup subscription **/ @@ -893,13 +896,13 @@ public void testFunctionAutomaticSubCleanup() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // validate pulsar source consumer has started on the topic - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); retryStrategically((test) -> { try { @@ -930,14 +933,14 @@ public void testFunctionAutomaticSubCleanup() throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); } @Test(timeOut = 20000) @@ -986,7 +989,7 @@ public void testMultiTopicFunction() throws Exception { assertTrue(retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic1).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic1).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } @@ -994,7 +997,7 @@ public void testMultiTopicFunction() throws Exception { assertTrue(retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic2).subscriptions.size() == 1; + return admin.topics().getStats(sourceTopic2).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } @@ -1009,9 +1012,9 @@ public void testMultiTopicFunction() throws Exception { assertTrue(retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic1).subscriptions.get( + SubscriptionStats subStats = admin.topics().getStats(sourceTopic1).getSubscriptions().get( InstanceUtils.getDefaultSubscriptionName(tenant, namespacePortion, functionName)); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -1019,9 +1022,9 @@ public void testMultiTopicFunction() throws Exception { assertTrue(retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic2).subscriptions.get( + SubscriptionStats subStats = admin.topics().getStats(sourceTopic2).getSubscriptions().get( InstanceUtils.getDefaultSubscriptionName(tenant, namespacePortion, functionName)); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -1049,7 +1052,7 @@ public void testMultiTopicFunction() throws Exception { assertTrue(retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic1).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic1).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } @@ -1057,7 +1060,7 @@ public void testMultiTopicFunction() throws Exception { assertTrue(retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic2).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic2).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java index 0d044fa0e33b4..b921266a60c10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java @@ -163,9 +163,9 @@ private void testPulsarSinkDLQ() throws Exception { try { TopicStats topicStats = admin.topics().getStats(sourceTopic); - return topicStats.subscriptions.containsKey(subscriptionName) - && topicStats.subscriptions.get(subscriptionName).consumers.size() == 1 - && topicStats.subscriptions.get(subscriptionName).consumers.get(0).availablePermits == 1000; + return topicStats.getSubscriptions().containsKey(subscriptionName) + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().size() == 1 + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().get(0).getAvailablePermits() == 1000; } catch (PulsarAdminException e) { return false; @@ -234,9 +234,9 @@ private void testPulsarSinkStats(String jarFilePathUrl) throws Exception { try { TopicStats topicStats = admin.topics().getStats(sourceTopic); - return topicStats.subscriptions.containsKey(subscriptionName) - && topicStats.subscriptions.get(subscriptionName).consumers.size() == 1 - && topicStats.subscriptions.get(subscriptionName).consumers.get(0).availablePermits == 523; + return topicStats.getSubscriptions().containsKey(subscriptionName) + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().size() == 1 + && topicStats.getSubscriptions().get(subscriptionName).getConsumers().get(0).getAvailablePermits() == 523; } catch (PulsarAdminException e) { return false; @@ -244,10 +244,10 @@ private void testPulsarSinkStats(String jarFilePathUrl) throws Exception { }, 50, 150); TopicStats topicStats = admin.topics().getStats(sourceTopic); - assertEquals(topicStats.subscriptions.size(), 1); - assertTrue(topicStats.subscriptions.containsKey(subscriptionName)); - assertEquals(topicStats.subscriptions.get(subscriptionName).consumers.size(), 1); - assertEquals(topicStats.subscriptions.get(subscriptionName).consumers.get(0).availablePermits, 523); + assertEquals(topicStats.getSubscriptions().size(), 1); + assertTrue(topicStats.getSubscriptions().containsKey(subscriptionName)); + assertEquals(topicStats.getSubscriptions().get(subscriptionName).getConsumers().size(), 1); + assertEquals(topicStats.getSubscriptions().get(subscriptionName).getConsumers().get(0).getAvailablePermits(), 523); // validate prometheus metrics empty String prometheusMetrics = PulsarFunctionTestUtils.getPrometheusMetrics(pulsar.getListenPortHTTP().get()); @@ -325,8 +325,8 @@ private void testPulsarSinkStats(String jarFilePathUrl) throws Exception { } retryStrategically((test) -> { try { - SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName); - return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs; + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + return subStats.getUnackedMessages() == 0 && subStats.getMsgThroughputOut() == totalMsgs; } catch (PulsarAdminException e) { return false; } @@ -407,14 +407,14 @@ private void testPulsarSinkStats(String jarFilePathUrl) throws Exception { retryStrategically((test) -> { try { - return admin.topics().getStats(sourceTopic).subscriptions.size() == 0; + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; } catch (PulsarAdminException e) { return false; } }, 50, 150); // make sure subscriptions are cleanup - assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 0); + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); tempDirectory.assertThatFunctionDownloadTempFilesHaveBeenDeleted(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSourceE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSourceE2ETest.java index af083633d9788..8029703a4990c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSourceE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSourceE2ETest.java @@ -64,7 +64,7 @@ private void testPulsarSourceStats(String jarFilePathUrl) throws Exception { retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic).publishers.size() == 1); + return (admin.topics().getStats(sinkTopic).getPublishers().size() == 1); } catch (PulsarAdminException e) { return false; } @@ -82,29 +82,29 @@ private void testPulsarSourceStats(String jarFilePathUrl) throws Exception { retryStrategically((test) -> { try { TopicStats sourceStats = admin.topics().getStats(sinkTopic2); - return sourceStats.publishers.size() == 1 - && sourceStats.publishers.get(0).metadata != null - && sourceStats.publishers.get(0).metadata.containsKey("id") - && sourceStats.publishers.get(0).metadata.get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); + return sourceStats.getPublishers().size() == 1 + && sourceStats.getPublishers().get(0).getMetadata() != null + && sourceStats.getPublishers().get(0).getMetadata().containsKey("id") + && sourceStats.getPublishers().get(0).getMetadata().get("id").equals(String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); } catch (PulsarAdminException e) { return false; } }, 50, 150); TopicStats sourceStats = admin.topics().getStats(sinkTopic2); - assertEquals(sourceStats.publishers.size(), 1); - assertNotNull(sourceStats.publishers.get(0).metadata); - assertTrue(sourceStats.publishers.get(0).metadata.containsKey("id")); - assertEquals(sourceStats.publishers.get(0).metadata.get("id"), String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); + assertEquals(sourceStats.getPublishers().size(), 1); + assertNotNull(sourceStats.getPublishers().get(0).getMetadata()); + assertTrue(sourceStats.getPublishers().get(0).getMetadata().containsKey("id")); + assertEquals(sourceStats.getPublishers().get(0).getMetadata().get("id"), String.format("%s/%s/%s", tenant, namespacePortion, sourceName)); retryStrategically((test) -> { try { - return (admin.topics().getStats(sinkTopic2).publishers.size() == 1) && (admin.topics().getInternalStats(sinkTopic2, false).numberOfEntries > 4); + return (admin.topics().getStats(sinkTopic2).getPublishers().size() == 1) && (admin.topics().getInternalStats(sinkTopic2, false).numberOfEntries > 4); } catch (PulsarAdminException e) { return false; } }, 50, 150); - assertEquals(admin.topics().getStats(sinkTopic2).publishers.size(), 1); + assertEquals(admin.topics().getStats(sinkTopic2).getPublishers().size(), 1); String prometheusMetrics = PulsarFunctionTestUtils.getPrometheusMetrics(pulsar.getListenPortHTTP().get()); log.info("prometheusMetrics: {}", prometheusMetrics); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/PartitionedTopicSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/PartitionedTopicSchemaTest.java index 61270263126d1..911a4ab82db0c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/PartitionedTopicSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/PartitionedTopicSchemaTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -51,7 +52,7 @@ protected void setup() throws Exception { isTcpLookup = true; super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index ee39f9b8b0167..62f18ecae3b48 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -64,8 +64,10 @@ import org.apache.pulsar.client.impl.schema.generic.GenericJsonRecord; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.KeyValueEncodingType; @@ -89,9 +91,10 @@ public void setup() throws Exception { super.internalSetup(); // Setup namespaces - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl(pulsar.getBrokerServiceUrl())); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Collections.singleton(CLUSTER_NAME)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton(CLUSTER_NAME)) + .build(); admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java index b4eb3e4622606..d6d96f79549de 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java @@ -38,8 +38,10 @@ 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.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; @@ -61,9 +63,10 @@ public void setup() throws Exception { super.internalSetup(); // Setup namespaces - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl(pulsar.getBrokerServiceUrl())); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Collections.singleton(CLUSTER_NAME)); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton(CLUSTER_NAME)) + .build(); admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaTypeCompatibilityCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaTypeCompatibilityCheckTest.java index 67be4ebbc717b..4fc57aff046a5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaTypeCompatibilityCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaTypeCompatibilityCheckTest.java @@ -29,8 +29,10 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.schema.Schemas; @@ -60,10 +62,11 @@ public void setup() throws Exception { super.internalSetup(); // Setup namespaces - admin.clusters().createCluster(CLUSTER_NAME, new ClusterDataImpl(pulsar.getBrokerServiceUrl())); + admin.clusters().createCluster(CLUSTER_NAME, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); - TenantInfoImpl tenantInfo = new TenantInfoImpl(); - tenantInfo.setAllowedClusters(Collections.singleton(CLUSTER_NAME)); + TenantInfo tenantInfo = TenantInfo.builder() + .allowedClusters(Collections.singleton(CLUSTER_NAME)) + .build(); admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo); admin.namespaces().createNamespace(namespaceName, Sets.newHashSet(CLUSTER_NAME)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java index 9e22219a70e41..78f33706d5355 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.authorization.AuthorizationService; 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.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -87,7 +88,7 @@ public void test() throws Exception { assertFalse(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); - admin.clusters().createCluster(configClusterName, new ClusterDataImpl()); + admin.clusters().createCluster(configClusterName, ClusterData.builder().build()); admin.tenants().createTenant("p1", new TenantInfoImpl(Sets.newHashSet("role1"), Sets.newHashSet("c1"))); waitForChange(); admin.namespaces().createNamespace("p1/c1/ns1"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java index a13a14e25e730..2019a9ebd9a33 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java @@ -407,7 +407,10 @@ public void producerBacklogQuotaExceededTest() throws Exception { admin.namespaces().createNamespace(namespace); admin.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("test")); admin.namespaces().setBacklogQuota(namespace, - new BacklogQuota(10, BacklogQuota.RetentionPolicy.producer_request_hold)); + BacklogQuota.builder() + .limitSize(10) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_request_hold) + .build()); final String topic = namespace + "/my-topic5"; final String subscription = "my-sub"; @@ -475,7 +478,10 @@ public void topicDoesNotExistTest() throws Exception { admin.namespaces().createNamespace(namespace); admin.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("test")); admin.namespaces().setAutoTopicCreation(namespace, - new AutoTopicCreationOverride(false, TopicType.NON_PARTITIONED.toString(), null)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(false) + .topicType(TopicType.NON_PARTITIONED.toString()) + .build()); final String topic = namespace + "/my-topic"; final String subscription = "my-sub"; diff --git a/pulsar-client-admin-api/pom.xml b/pulsar-client-admin-api/pom.xml index 113040546b3b5..3e3ce27994a14 100644 --- a/pulsar-client-admin-api/pom.xml +++ b/pulsar-client-admin-api/pom.xml @@ -33,7 +33,6 @@ Pulsar Client Admin :: API - ${project.groupId} pulsar-client-api @@ -43,9 +42,7 @@ org.slf4j slf4j-api - provided - diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/utils/ReflectionUtils.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/utils/ReflectionUtils.java index b14a0ee875821..b6313e58876ae 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/utils/ReflectionUtils.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/utils/ReflectionUtils.java @@ -24,11 +24,18 @@ import lombok.experimental.UtilityClass; @UtilityClass -class ReflectionUtils { +public class ReflectionUtils { interface SupplierWithException { T get() throws Exception; } + public static T newBuilder(String className) { + return catchExceptions( + () -> (T) ReflectionUtils.getStaticMethod( + className, "builder", null) + .invoke(null, null)); + } + static T catchExceptions(SupplierWithException s) { try { return s.get(); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AuthPolicies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AuthPolicies.java index de882d160dcd6..6ca81170d6946 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AuthPolicies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AuthPolicies.java @@ -18,43 +18,26 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.HashMap; import java.util.Map; -import java.util.Objects; import java.util.Set; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; /** * Authentication policies. */ -public class AuthPolicies { - @SuppressWarnings("checkstyle:MemberName") - public final Map> namespace_auth; - @SuppressWarnings("checkstyle:MemberName") - public final Map>> destination_auth; - @SuppressWarnings("checkstyle:MemberName") - public final Map> subscription_auth_roles; +public interface AuthPolicies { + Map> getNamespaceAuthentication(); + Map>> getTopicAuthentication(); + Map> getSubscriptionAuthentication(); - public AuthPolicies() { - namespace_auth = new HashMap<>(); - destination_auth = new HashMap<>(); - subscription_auth_roles = new HashMap<>(); + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.client.admin.internal.data.AuthPoliciesImpl"); } - @Override - public int hashCode() { - return Objects.hash(namespace_auth, destination_auth, - subscription_auth_roles); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof AuthPolicies) { - AuthPolicies other = (AuthPolicies) obj; - return Objects.equals(namespace_auth, other.namespace_auth) - && Objects.equals(destination_auth, other.destination_auth) - && Objects.equals(subscription_auth_roles, other.subscription_auth_roles); - } - - return false; + interface Builder { + AuthPolicies build(); + Builder namespaceAuthentication(Map> namespaceAuthentication); + Builder topicAuthentication(Map>> topicAuthentication); + Builder subscriptionAuthentication(Map> subscriptionAuthentication); } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyData.java index 1c19807dcc6e9..c9b6073b0801f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyData.java @@ -19,13 +19,24 @@ package org.apache.pulsar.common.policies.data; import java.util.Map; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; public interface AutoFailoverPolicyData { - AutoFailoverPolicyType getPolicy_type(); + AutoFailoverPolicyType getPolicyType(); Map getParameters(); - void setPolicy_type(AutoFailoverPolicyType policyType); + void validate(); - void setParameters(Map parameters); + interface Builder { + Builder policyType(AutoFailoverPolicyType policyType); + + Builder parameters(Map parameters); + + AutoFailoverPolicyData build(); + } + + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl"); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoSubscriptionCreationOverride.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoSubscriptionCreationOverride.java index 2a57b751b5541..bb444aa798517 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoSubscriptionCreationOverride.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoSubscriptionCreationOverride.java @@ -1,51 +1,39 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.policies.data; - -import java.util.Objects; -import lombok.ToString; - -/** - * Override of autoSubscriptionCreation settings on a namespace level. - */ -@ToString -public class AutoSubscriptionCreationOverride { - public boolean allowAutoSubscriptionCreation; - - public AutoSubscriptionCreationOverride() { - } - - public AutoSubscriptionCreationOverride(boolean allowAutoSubscriptionCreation) { - this.allowAutoSubscriptionCreation = allowAutoSubscriptionCreation; - } - - @Override - public int hashCode() { - return Objects.hash(allowAutoSubscriptionCreation); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof AutoSubscriptionCreationOverride) { - AutoSubscriptionCreationOverride other = (AutoSubscriptionCreationOverride) obj; - return Objects.equals(this.allowAutoSubscriptionCreation, other.allowAutoSubscriptionCreation); - } - return false; - } -} +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data; + +import org.apache.pulsar.common.policies.data.impl.AutoSubscriptionCreationOverrideImpl; + +/** + * Override of autoSubscriptionCreation settings on a namespace level. + */ +public interface AutoSubscriptionCreationOverride { + + boolean isAllowAutoSubscriptionCreation(); + + static Builder builder() { + return AutoSubscriptionCreationOverrideImpl.builder(); + } + + interface Builder { + Builder allowAutoSubscriptionCreation(boolean allowAutoSubscriptionCreation); + + AutoSubscriptionCreationOverride build(); + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java index 750b771182b15..1ad981c5be7b1 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java @@ -18,66 +18,29 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; -import lombok.ToString; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; /** * Override of autoTopicCreation settings on a namespace level. */ -@ToString -public class AutoTopicCreationOverride { - public boolean allowAutoTopicCreation; - public String topicType; - public Integer defaultNumPartitions; +public interface AutoTopicCreationOverride { + boolean isAllowAutoTopicCreation(); - public AutoTopicCreationOverride() { - } + String getTopicType(); - public AutoTopicCreationOverride(boolean allowAutoTopicCreation, String topicType, - Integer defaultNumPartitions) { - this.allowAutoTopicCreation = allowAutoTopicCreation; - this.topicType = topicType; - this.defaultNumPartitions = defaultNumPartitions; - } + Integer getDefaultNumPartitions(); - @Override - public int hashCode() { - return Objects.hash(allowAutoTopicCreation, topicType, defaultNumPartitions); - } + interface Builder { + Builder allowAutoTopicCreation(boolean allowTopicCreation); - @Override - public boolean equals(Object obj) { - if (obj instanceof AutoTopicCreationOverride) { - AutoTopicCreationOverride other = (AutoTopicCreationOverride) obj; - return Objects.equals(this.allowAutoTopicCreation, other.allowAutoTopicCreation) - && Objects.equals(this.topicType, other.topicType) - && Objects.equals(this.defaultNumPartitions, other.defaultNumPartitions); - } - return false; - } + Builder topicType(String topicType); + + Builder defaultNumPartitions(Integer defaultNumPartition); - public static boolean isValidOverride(AutoTopicCreationOverride override) { - if (override == null) { - return false; - } - if (override.allowAutoTopicCreation) { - if (!TopicType.isValidTopicType(override.topicType)) { - return false; - } - if (TopicType.PARTITIONED.toString().equals(override.topicType)) { - if (override.defaultNumPartitions == null) { - return false; - } - if (!(override.defaultNumPartitions > 0)) { - return false; - } - } else if (TopicType.NON_PARTITIONED.toString().equals(override.topicType)) { - if (override.defaultNumPartitions != null) { - return false; - } - } - } - return true; + AutoTopicCreationOverride build(); } + static Builder builder() { + return AutoTopicCreationOverrideImpl.builder(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BacklogQuota.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BacklogQuota.java index 0bb39d27c0992..d4b5c4bba1c5b 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BacklogQuota.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BacklogQuota.java @@ -18,8 +18,7 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; -import lombok.ToString; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; /** * Unit of a backlog quota configuration for a scoped resource in a Pulsar instance. @@ -27,81 +26,42 @@ *

A scoped resource is identified by a {@link BacklogQuotaType} enumeration type which is containing two attributes: * limit representing a quota limit in bytes and policy for backlog retention policy. */ -@ToString -public class BacklogQuota { - private long limitSize; - // backlog quota by time in second - private int limitTime; - private RetentionPolicy policy; +public interface BacklogQuota { /** * Gets quota limit in size. * * @return quota limit in bytes */ - public long getLimitSize() { - return limitSize; - } + long getLimitSize(); /** * Gets quota limit in time. * * @return quota limit in second */ - public int getLimitTime() { - return limitTime; - } - - public RetentionPolicy getPolicy() { - return policy; - } + int getLimitTime(); - /** - * Sets quota limit in bytes. - * - * @param limitSize - * quota limit in bytes - */ - public void setLimitSize(long limitSize) { - this.limitSize = limitSize; - } + RetentionPolicy getPolicy(); - public void setPolicy(RetentionPolicy policy) { - this.policy = policy; - } + interface Builder { + Builder limitSize(long limitSize); - protected BacklogQuota() { - } + Builder limitTime(int limitTime); - public BacklogQuota(long limitSize, RetentionPolicy policy) { - this(limitSize, -1, policy); - } + Builder retentionPolicy(RetentionPolicy retentionPolicy); - public BacklogQuota(long limitSize, int limitTime, RetentionPolicy policy) { - this.limitTime = limitTime; - this.limitSize = limitSize; - this.policy = policy; + BacklogQuota build(); } - @Override - public int hashCode() { - return Objects.hash(Long.valueOf(limitSize), Long.valueOf(limitTime), policy); + static Builder builder() { + return BacklogQuotaImpl.builder(); } - @Override - public boolean equals(Object obj) { - if (obj instanceof BacklogQuota) { - BacklogQuota other = (BacklogQuota) obj; - return Objects.equals(limitSize, other.limitSize) && Objects.equals(limitTime, other.limitTime) - && Objects.equals(policy, other.policy); - } - return false; - }; - /** * Identifier to a backlog quota configuration (an instance of {@link BacklogQuota}). */ - public enum BacklogQuotaType { + enum BacklogQuotaType { destination_storage, message_age, } @@ -109,7 +69,7 @@ public enum BacklogQuotaType { /** * Enumeration type determines how to retain backlog against the resource shortages. */ - public enum RetentionPolicy { + enum RetentionPolicy { /** Policy which holds producer's send request until the resource becomes available (or holding times out). */ producer_request_hold, diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieAffinityGroupData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieAffinityGroupData.java index 1acc2bc8899f1..c0fd572b99597 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieAffinityGroupData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieAffinityGroupData.java @@ -18,42 +18,26 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; +import org.apache.pulsar.common.policies.data.impl.BookieAffinityGroupDataImpl; /** * Description of a BookKeeper's affinity group. */ -public class BookieAffinityGroupData { +public interface BookieAffinityGroupData { - public String bookkeeperAffinityGroupPrimary; - public String bookkeeperAffinityGroupSecondary; + String getBookkeeperAffinityGroupPrimary(); - public BookieAffinityGroupData() { - } + String getBookkeeperAffinityGroupSecondary(); - public BookieAffinityGroupData(String bookkeeperAffinityGroupPrimary, String bookkeeperAffinityGroupSecondary) { - this.bookkeeperAffinityGroupPrimary = bookkeeperAffinityGroupPrimary; - this.bookkeeperAffinityGroupSecondary = bookkeeperAffinityGroupSecondary; - } + interface Builder { + Builder bookkeeperAffinityGroupPrimary(String bookkeeperAffinityGroupPrimary); - @Override - public int hashCode() { - return Objects.hash(bookkeeperAffinityGroupPrimary, bookkeeperAffinityGroupSecondary); - } + Builder bookkeeperAffinityGroupSecondary(String bookkeeperAffinityGroupSecondary); - @Override - public boolean equals(Object obj) { - if (obj instanceof BookieAffinityGroupData) { - BookieAffinityGroupData other = (BookieAffinityGroupData) obj; - return Objects.equals(bookkeeperAffinityGroupPrimary, other.bookkeeperAffinityGroupPrimary) - && Objects.equals(bookkeeperAffinityGroupSecondary, other.bookkeeperAffinityGroupSecondary); - } - return false; + BookieAffinityGroupData build(); } - @Override - public String toString() { - return String.format("bookkeeperAffinityGroupPrimary=%s bookkeeperAffinityGroupSecondary=%s", - bookkeeperAffinityGroupPrimary, bookkeeperAffinityGroupSecondary); + static Builder builder() { + return BookieAffinityGroupDataImpl.builder(); } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieInfo.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieInfo.java index 6d5eb79091cc8..808cbbd0b41a7 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieInfo.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookieInfo.java @@ -18,17 +18,22 @@ */ package org.apache.pulsar.common.policies.data; -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.impl.BookieInfoImpl; /** * Bookie information. */ -@Data -@AllArgsConstructor -@NoArgsConstructor -public class BookieInfo { - private String rack; - private String hostname; +public interface BookieInfo { + String getRack(); + String getHostname(); + + interface Builder { + Builder rack(String rack); + Builder hostname(String hostname); + BookieInfo build(); + } + + static Builder builder() { + return BookieInfoImpl.builder(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookiesClusterInfo.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookiesClusterInfo.java index e16012e65cbc8..818f69d8201ac 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookiesClusterInfo.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BookiesClusterInfo.java @@ -19,22 +19,21 @@ package org.apache.pulsar.common.policies.data; import java.util.List; -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.NoArgsConstructor; -import lombok.ToString; +import org.apache.pulsar.common.policies.data.impl.BookiesClusterInfoImpl; /** * Raw bookies information. */ -@Data -@NoArgsConstructor -@AllArgsConstructor -@ToString -public class BookiesClusterInfo { +public interface BookiesClusterInfo { - private static final long serialVersionUID = 0L; + List getBookies(); - private List bookies; + interface Builder { + Builder bookies(List bookies); + BookiesClusterInfo build(); + } + static Builder builder() { + return BookiesClusterInfoImpl.builder(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerInfo.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerInfo.java index 7581049d461cd..c62f3707c733f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerInfo.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerInfo.java @@ -18,16 +18,20 @@ */ package org.apache.pulsar.common.policies.data; -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.impl.BrokerInfoImpl; /** * Broker Information. */ -@Data -@AllArgsConstructor -@NoArgsConstructor -public class BrokerInfo { - private String serviceUrl; +public interface BrokerInfo { + String getServiceUrl(); + + interface Builder { + Builder serviceUrl(String serviceUrl); + BrokerInfo build(); + } + + static Builder builder() { + return BrokerInfoImpl.builder(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationData.java index 3ea8e42a53b1b..a53ffd3d5f1fe 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationData.java @@ -19,6 +19,7 @@ package org.apache.pulsar.common.policies.data; import java.util.List; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; public interface BrokerNamespaceIsolationData { String getBrokerName(); @@ -29,11 +30,22 @@ public interface BrokerNamespaceIsolationData { List getNamespaceRegex(); - void setBrokerName(String brokerName); + interface Builder { + Builder brokerName(String brokerName); - void setPolicyName(String policyName); + Builder policyName(String policyName); + + Builder primary(boolean isPrimary); + + Builder namespaceRegex(List namespaceRegex); + + BrokerNamespaceIsolationData build(); + } + + static Builder builder() { + return ReflectionUtils.newBuilder( + "org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl"); + } - void setPrimary(boolean isPrimary); - void setNamespaceRegex(List namespaceRegex); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerStatus.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerStatus.java index fb3498edb665d..82a2544912d49 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerStatus.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BrokerStatus.java @@ -18,54 +18,29 @@ */ package org.apache.pulsar.common.policies.data; -import lombok.EqualsAndHashCode; +import org.apache.pulsar.common.policies.data.impl.BrokerStatusImpl; /** * Information about the broker status. */ -@EqualsAndHashCode -public class BrokerStatus implements Comparable { - private String brokerAddress; - private boolean active; - private int loadFactor; +public interface BrokerStatus extends Comparable { + String getBrokerAddress(); - public BrokerStatus(String lookupServiceAddress, boolean active, int loadFactor) { - this.brokerAddress = lookupServiceAddress; - this.active = active; - this.loadFactor = loadFactor; - } + boolean isActive(); - public boolean isActive() { - return this.active; - } + int getLoadFactor(); - public int getLoadFactor() { - return this.loadFactor; - } + interface Builder { + Builder brokerAddress(String brokerAddress); - public String getBrokerAddress() { - return this.brokerAddress; - } + Builder active(boolean active); - public void setActive(boolean active) { - this.active = active; - } - - public void setLoadFactor(int loadFactor) { - this.loadFactor = loadFactor; - } + Builder loadFactor(int loadFactor); - @Override - public int compareTo(BrokerStatus other) { - int result = Integer.compare(this.loadFactor, other.loadFactor); - if (result == 0) { - result = this.brokerAddress.compareTo(other.brokerAddress); - } - return result; + BrokerStatus build(); } - @Override - public String toString() { - return String.format("[brokerAddress=%s, active=%s, loadFactor=%s]", brokerAddress, active, loadFactor); + static Builder builder() { + return BrokerStatusImpl.builder(); } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BundlesData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BundlesData.java index 3540a258ec71a..450d17c749d53 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BundlesData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/BundlesData.java @@ -19,61 +19,23 @@ package org.apache.pulsar.common.policies.data; import java.util.List; -import java.util.Objects; -import lombok.ToString; +import org.apache.pulsar.common.policies.data.impl.BundlesDataImpl; /** * Holder for bundles. */ -@ToString -public class BundlesData { - public List boundaries; - public int numBundles; +public interface BundlesData { + List getBoundaries(); + int getNumBundles(); - public BundlesData() { - this.boundaries = null; - this.numBundles = 0; + interface Builder { + Builder boundaries(List boundaries); + Builder numBundles(int numBundles); + BundlesData build(); } - public BundlesData(List boundaries) { - this.boundaries = boundaries; - this.numBundles = boundaries.size() - 1; - } - - public BundlesData(int numBundles) { - this.boundaries = null; - this.numBundles = numBundles; - } - - public List getBoundaries() { - return boundaries; - } - - public void setBoundaries(List boundaries) { - this.boundaries = boundaries; - } - - public int getNumBundles() { - return numBundles; - } - - public void setNumBundles(int numBundles) { - this.numBundles = numBundles; - } - - @Override - public int hashCode() { - return Objects.hash(boundaries); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof BundlesData) { - BundlesData other = (BundlesData) obj; - return Objects.equals(boundaries, other.boundaries); - } - - return false; + static Builder builder() { + return BundlesDataImpl.builder(); } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java index f6685c5e6993c..61a90a592a70d 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java @@ -19,6 +19,7 @@ package org.apache.pulsar.common.policies.data; import java.util.LinkedHashSet; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; import org.apache.pulsar.client.api.ProxyProtocol; public interface ClusterData { @@ -26,59 +27,77 @@ public interface ClusterData { String getServiceUrlTls(); - ClusterData setServiceUrl(String serviceUrl); - - ClusterData setServiceUrlTls(String serviceUrlTls); - String getBrokerServiceUrl(); - ClusterData setBrokerServiceUrl(String brokerServiceUrl); - String getBrokerServiceUrlTls(); - ClusterData setBrokerServiceUrlTls(String brokerServiceUrlTls); - String getProxyServiceUrl(); - ClusterData setProxyServiceUrl(String proxyServiceUrl); - ProxyProtocol getProxyProtocol(); - ClusterData setProxyProtocol(ProxyProtocol proxyProtocol); - LinkedHashSet getPeerClusterNames(); String getAuthenticationPlugin(); - ClusterData setAuthenticationPlugin(String authenticationPlugin); - String getAuthenticationParameters(); - ClusterData setAuthenticationParameters(String authenticationParameters); - - ClusterData setPeerClusterNames(LinkedHashSet peerClusterNames); - - ClusterData setBrokerClientTlsEnabled(boolean enabled); boolean isBrokerClientTlsEnabled(); - ClusterData setTlsAllowInsecureConnection(boolean enabled); boolean isTlsAllowInsecureConnection(); - ClusterData setBrokerClientTlsEnabledWithKeyStore(boolean enabled); boolean isBrokerClientTlsEnabledWithKeyStore(); - ClusterData setBrokerClientTlsTrustStoreType(String trustStoreType); String getBrokerClientTlsTrustStoreType(); - ClusterData setBrokerClientTlsTrustStore(String tlsTrustStore); String getBrokerClientTlsTrustStore(); - ClusterData setBrokerClientTlsTrustStorePassword(String trustStorePassword); String getBrokerClientTlsTrustStorePassword(); - ClusterData setBrokerClientTrustCertsFilePath(String trustCertsFilePath); String getBrokerClientTrustCertsFilePath(); - ClusterData setListenerName(String listenerName); String getListenerName(); + + interface Builder { + Builder serviceUrl(String serviceUrl); + + Builder serviceUrlTls(String serviceUrlTls); + + Builder brokerServiceUrl(String brokerServiceUrl); + + Builder brokerServiceUrlTls(String brokerServiceUrlTls); + + Builder proxyServiceUrl(String proxyServiceUrl); + + Builder proxyProtocol(ProxyProtocol proxyProtocol); + + Builder authenticationPlugin(String authenticationPlugin); + + Builder authenticationParameters(String authenticationParameters); + + Builder peerClusterNames(LinkedHashSet peerClusterNames); + + Builder brokerClientTlsEnabled(boolean enabled); + + Builder tlsAllowInsecureConnection(boolean enabled); + + Builder brokerClientTlsEnabledWithKeyStore(boolean enabled); + + Builder brokerClientTlsTrustStoreType(String trustStoreType); + + Builder brokerClientTlsTrustStore(String tlsTrustStore); + + Builder brokerClientTlsTrustStorePassword(String trustStorePassword); + + Builder brokerClientTrustCertsFilePath(String trustCertsFilePath); + + Builder listenerName(String listenerName); + + ClusterData build(); + } + + Builder clone(); + + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.common.policies.data.ClusterDataImpl"); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index 4bba99798f6ed..7204af616e2d8 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -20,141 +20,62 @@ import java.util.List; import java.util.Map; -import java.util.Objects; /** * Consumer statistics. */ -public class ConsumerStats { +public interface ConsumerStats { /** Total rate of messages delivered to the consumer (msg/s). */ - public double msgRateOut; + double getMsgRateOut(); /** Total throughput delivered to the consumer (bytes/s). */ - public double msgThroughputOut; + double getMsgThroughputOut(); /** Total bytes delivered to consumer (bytes). */ - public long bytesOutCounter; + long getBytesOutCounter(); /** Total messages delivered to consumer (msg). */ - public long msgOutCounter; + long getMsgOutCounter(); /** Total rate of messages redelivered by this consumer (msg/s). */ - public double msgRateRedeliver; - - /** - * Total chunked messages dispatched. - * @deprecated use {@link chunkedMessageRate)} - */ - @Deprecated - public double chuckedMessageRate; + double getMsgRateRedeliver(); /** Total chunked messages dispatched. */ - public double chunkedMessageRate; + double getChunkedMessageRate(); /** Name of the consumer. */ - public String consumerName; + String getConsumerName(); /** Number of available message permits for the consumer. */ - public int availablePermits; + int getAvailablePermits(); /** Number of unacknowledged messages for the consumer. */ - public int unackedMessages; + int getUnackedMessages(); /** Number of average messages per entry for the consumer consumed. */ - public int avgMessagesPerEntry; + int getAvgMessagesPerEntry(); /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ - public boolean blockedConsumerOnUnackedMsgs; + boolean isBlockedConsumerOnUnackedMsgs(); /** The read position of the cursor when the consumer joining. */ - public String readPositionWhenJoining; + String getReadPositionWhenJoining(); /** Address of this consumer. */ - private int addressOffset = -1; - private int addressLength; + String getAddress(); /** Timestamp of connection. */ - private int connectedSinceOffset = -1; - private int connectedSinceLength; + String getConnectedSince(); /** Client library version. */ - private int clientVersionOffset = -1; - private int clientVersionLength; + String getClientVersion(); - public long lastAckedTimestamp; - public long lastConsumedTimestamp; + long getLastAckedTimestamp(); + long getLastConsumedTimestamp(); /** Hash ranges assigned to this consumer if is Key_Shared sub mode. **/ - public List keyHashRanges; + List getKeyHashRanges(); /** Metadata (key/value strings) associated with this consumer. */ - public Map metadata; - - /** - * In order to prevent multiple string object allocation under stats: create a string-buffer - * that stores data for all string place-holders. - */ - private StringBuilder stringBuffer = new StringBuilder(); - - public ConsumerStats add(ConsumerStats stats) { - Objects.requireNonNull(stats); - this.msgRateOut += stats.msgRateOut; - this.msgThroughputOut += stats.msgThroughputOut; - this.bytesOutCounter += stats.bytesOutCounter; - this.msgOutCounter += stats.msgOutCounter; - this.msgRateRedeliver += stats.msgRateRedeliver; - this.availablePermits += stats.availablePermits; - this.unackedMessages += stats.unackedMessages; - this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; - this.readPositionWhenJoining = stats.readPositionWhenJoining; - return this; - } - - public String getAddress() { - return addressOffset == -1 ? null : stringBuffer.substring(addressOffset, addressOffset + addressLength); - } - - public void setAddress(String address) { - if (address == null) { - this.addressOffset = -1; - return; - } - this.addressOffset = this.stringBuffer.length(); - this.addressLength = address.length(); - this.stringBuffer.append(address); - } - - public String getConnectedSince() { - return connectedSinceOffset == -1 ? null - : stringBuffer.substring(connectedSinceOffset, connectedSinceOffset + connectedSinceLength); - } - - public void setConnectedSince(String connectedSince) { - if (connectedSince == null) { - this.connectedSinceOffset = -1; - return; - } - this.connectedSinceOffset = this.stringBuffer.length(); - this.connectedSinceLength = connectedSince.length(); - this.stringBuffer.append(connectedSince); - } - - public String getClientVersion() { - return clientVersionOffset == -1 ? null - : stringBuffer.substring(clientVersionOffset, clientVersionOffset + clientVersionLength); - } - - public void setClientVersion(String clientVersion) { - if (clientVersion == null) { - this.clientVersionOffset = -1; - return; - } - this.clientVersionOffset = this.stringBuffer.length(); - this.clientVersionLength = clientVersion.length(); - this.stringBuffer.append(clientVersion); - } - - public String getReadPositionWhenJoining() { - return readPositionWhenJoining; - } + Map getMetadata(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DelayedDeliveryPolicies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DelayedDeliveryPolicies.java index 797f090d83e7c..17f1ab71ceb52 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DelayedDeliveryPolicies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DelayedDeliveryPolicies.java @@ -18,21 +18,22 @@ */ package org.apache.pulsar.common.policies.data; -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.Getter; -import lombok.NoArgsConstructor; -import lombok.Setter; +import org.apache.pulsar.common.policies.data.impl.DelayedDeliveryPoliciesImpl; /** * Definition of the delayed delivery policy. */ -@Data -@AllArgsConstructor -@Setter -@Getter -@NoArgsConstructor -public class DelayedDeliveryPolicies { - private long tickTime; - private boolean active; +public interface DelayedDeliveryPolicies { + long getTickTime(); + boolean isActive(); + + interface Builder { + Builder tickTime(long tickTime); + Builder active(boolean active); + DelayedDeliveryPolicies build(); + } + + static Builder builder() { + return DelayedDeliveryPoliciesImpl.builder(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DispatchRate.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DispatchRate.java index e9f5f8e413ca7..a8bc65bd0ed73 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DispatchRate.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DispatchRate.java @@ -18,55 +18,28 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; -import lombok.ToString; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; /** * Dispatch rate. */ -@ToString -public class DispatchRate { +public interface DispatchRate { - public int dispatchThrottlingRateInMsg = -1; - public long dispatchThrottlingRateInByte = -1; - public boolean relativeToPublishRate = false; /* throttles dispatch relatively publish-rate */ - public int ratePeriodInSecond = 1; /* by default dispatch-rate will be calculate per 1 second */ + int getDispatchThrottlingRateInMsg(); + long getDispatchThrottlingRateInByte(); + boolean isRelativeToPublishRate(); /* throttles dispatch relatively publish-rate */ + int getRatePeriodInSecond(); /* by default dispatch-rate will be calculate per 1 second */ - public DispatchRate() { - super(); - this.dispatchThrottlingRateInMsg = -1; - this.dispatchThrottlingRateInByte = -1; - this.ratePeriodInSecond = 1; - } - - public DispatchRate(int dispatchThrottlingRateInMsg, long dispatchThrottlingRateInByte, - int ratePeriodInSecond) { - super(); - this.dispatchThrottlingRateInMsg = dispatchThrottlingRateInMsg; - this.dispatchThrottlingRateInByte = dispatchThrottlingRateInByte; - this.ratePeriodInSecond = ratePeriodInSecond; - } - - public DispatchRate(int dispatchThrottlingRateInMsg, long dispatchThrottlingRateInByte, - int ratePeriodInSecond, boolean relativeToPublishRate) { - this(dispatchThrottlingRateInMsg, dispatchThrottlingRateInByte, ratePeriodInSecond); - this.relativeToPublishRate = relativeToPublishRate; - } + interface Builder { + Builder dispatchThrottlingRateInMsg(int dispatchThrottlingRateMsg); + Builder dispatchThrottlingRateInByte(long dispatchThrottlingRateBytes); + Builder relativeToPublishRate(boolean relativeToPublishRate); + Builder ratePeriodInSecond(int ratePeriodInSeconds); - @Override - public int hashCode() { - return Objects.hash(dispatchThrottlingRateInMsg, dispatchThrottlingRateInByte, - ratePeriodInSecond); + DispatchRate build(); } - @Override - public boolean equals(Object obj) { - if (obj instanceof DispatchRate) { - DispatchRate rate = (DispatchRate) obj; - return Objects.equals(dispatchThrottlingRateInMsg, rate.dispatchThrottlingRateInMsg) - && Objects.equals(dispatchThrottlingRateInByte, rate.dispatchThrottlingRateInByte) - && Objects.equals(ratePeriodInSecond, rate.ratePeriodInSecond); - } - return false; + static Builder builder() { + return DispatchRateImpl.builder(); } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FailureDomain.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FailureDomain.java index b92717cc7d6be..401acecec29ea 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FailureDomain.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FailureDomain.java @@ -19,9 +19,17 @@ package org.apache.pulsar.common.policies.data; import java.util.Set; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; public interface FailureDomain { Set getBrokers(); - void setBrokers(Set brokers); + interface Builder { + Builder brokers(Set brokers); + FailureDomain build(); + } + + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.common.policies.data.FailureDomainImpl"); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStats.java index 03714476e3d08..bd087b0103670 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStats.java @@ -22,8 +22,4 @@ public interface FunctionInstanceStats { int getInstanceId(); FunctionInstanceStatsData getMetrics(); - - void setInstanceId(int instanceId); - - void setMetrics(FunctionInstanceStatsData metrics); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsData.java index 5d8d558051b35..7832e4c3993d5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsData.java @@ -18,16 +18,12 @@ */ package org.apache.pulsar.common.policies.data; +import java.util.Map; + public interface FunctionInstanceStatsData extends FunctionInstanceStatsDataBase { FunctionInstanceStatsDataBase getOneMin(); Long getLastInvocation(); - java.util.Map getUserMetrics(); - - void setOneMin(FunctionInstanceStatsDataBase oneMin); - - void setLastInvocation(Long lastInvocation); - - void setUserMetrics(java.util.Map userMetrics); + Map getUserMetrics(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataBase.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataBase.java index 578e19e3a1288..759f5f9b33bb0 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataBase.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataBase.java @@ -28,14 +28,4 @@ public interface FunctionInstanceStatsDataBase { long getUserExceptionsTotal(); Double getAvgProcessLatency(); - - void setReceivedTotal(long receivedTotal); - - void setProcessedSuccessfullyTotal(long processedSuccessfullyTotal); - - void setSystemExceptionsTotal(long systemExceptionsTotal); - - void setUserExceptionsTotal(long userExceptionsTotal); - - void setAvgProcessLatency(Double avgProcessLatency); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionStats.java index bf07d53e02600..5637c72375be4 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/FunctionStats.java @@ -21,8 +21,6 @@ import java.util.List; public interface FunctionStats { - void addInstance(FunctionInstanceStats functionInstanceStats); - FunctionStats calculateOverall(); long getReceivedTotal(); @@ -39,21 +37,5 @@ public interface FunctionStats { Long getLastInvocation(); - List getInstances(); - - void setReceivedTotal(long receivedTotal); - - void setProcessedSuccessfullyTotal(long processedSuccessfullyTotal); - - void setSystemExceptionsTotal(long systemExceptionsTotal); - - void setUserExceptionsTotal(long userExceptionsTotal); - - void setAvgProcessLatency(Double avgProcessLatency); - - void setOneMin(FunctionInstanceStatsDataBase oneMin); - - void setLastInvocation(Long lastInvocation); - - void setInstances(List instances); + List getInstances(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java index 6feb0d71de7cc..24cf030e761aa 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java @@ -19,6 +19,7 @@ package org.apache.pulsar.common.policies.data; import java.util.List; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; public interface NamespaceIsolationData { @@ -28,14 +29,23 @@ public interface NamespaceIsolationData { List getSecondary(); - AutoFailoverPolicyData getAuto_failover_policy(); + AutoFailoverPolicyData getAutoFailoverPolicy(); - void setNamespaces(List namespaces); + void validate(); - void setPrimary(List primary); + interface Builder { + Builder namespaces(List namespaces); - void setSecondary(List secondary); + Builder primary(List primary); - void setAuto_failover_policy(AutoFailoverPolicyData autoFailoverPolicyData); + Builder secondary(List secondary); + Builder autoFailoverPolicy(AutoFailoverPolicyData autoFailoverPolicyData); + + NamespaceIsolationData build(); + } + + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl"); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentPublisherStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentPublisherStats.java index 74e17524c8548..fd6622980d11f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentPublisherStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentPublisherStats.java @@ -18,22 +18,13 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; - /** * Non-persistent publisher statistics. */ -public class NonPersistentPublisherStats extends PublisherStats { +public interface NonPersistentPublisherStats extends PublisherStats { /** * for non-persistent topic: broker drops msg if publisher publishes messages more than configured max inflight * messages per connection. **/ - public double msgDropRate; - - public NonPersistentPublisherStats add(NonPersistentPublisherStats stats) { - Objects.requireNonNull(stats); - super.add(stats); - this.msgDropRate += stats.msgDropRate; - return this; - } + double getMsgDropRate(); } \ No newline at end of file diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java index d27813c3b272f..a506aa51d87ef 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java @@ -18,22 +18,13 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; - /** * Statistics for a non-persistent replicator. */ -public class NonPersistentReplicatorStats extends ReplicatorStats { +public interface NonPersistentReplicatorStats extends ReplicatorStats { /** * for non-persistent topic: broker drops msg for replicator if replicator connection is not writable. **/ - public double msgDropRate; - - public NonPersistentReplicatorStats add(NonPersistentReplicatorStats stats) { - Objects.requireNonNull(stats); - super.add(stats); - this.msgDropRate += stats.msgDropRate; - return this; - } + double getMsgDropRate(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentSubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentSubscriptionStats.java index 8d06d41991c21..515f0a5e8dd20 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentSubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentSubscriptionStats.java @@ -18,30 +18,14 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; - /** * Statistics for subscription to non-persistent topics. */ -public class NonPersistentSubscriptionStats extends SubscriptionStats { +public interface NonPersistentSubscriptionStats extends SubscriptionStats { /** * for non-persistent topic: broker drops msg for subscription if none of the consumer available for message * delivery. **/ - public double msgDropRate; - - public void reset() { - super.reset(); - msgDropRate = 0; - } - - // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current - // stats - public NonPersistentSubscriptionStats add(NonPersistentSubscriptionStats stats) { - Objects.requireNonNull(stats); - super.add(stats); - this.msgDropRate += stats.msgDropRate; - return this; - } + double getMsgDropRate(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentTopicStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentTopicStats.java index fd06101d7facc..2c952e8d2988d 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentTopicStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentTopicStats.java @@ -18,62 +18,26 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.TreeMap; /** * Statistics for a non-persistent topic. */ -public class NonPersistentTopicStats extends TopicStats { +public interface NonPersistentTopicStats extends TopicStats { /** * for non-persistent topic: broker drops msg if publisher publishes messages more than configured max inflight * messages per connection. **/ - public double msgDropRate; + double getMsgDropRate(); /** List of connected publishers on this topic w/ their stats. */ - public List publishers; + List getPublishers(); /** Map of subscriptions with their individual statistics. */ - public Map subscriptions; + Map getSubscriptions(); /** Map of replication statistics by remote cluster context. */ - public Map replication; - - public NonPersistentTopicStats() { - this.publishers = new ArrayList<>(); - this.subscriptions = new HashMap<>(); - this.replication = new TreeMap<>(); - } - - public void reset() { - super.reset(); - this.msgDropRate = 0; - } - - // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current - // stats. - public NonPersistentTopicStats add(NonPersistentTopicStats stats) { - Objects.requireNonNull(stats); - super.add(stats); - this.msgDropRate += stats.msgDropRate; - return this; - } - - public List getPublishers() { - return this.publishers; - } - - public Map getSubscriptions() { - return this.subscriptions; - } - - public Map getReplication() { - return this.replication; - } + Map getReplication(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java index 002e6f9007198..f99258aeb79d2 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.common.policies.data; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; + public interface OffloadPolicies { String getOffloadersDirectory(); @@ -75,59 +77,68 @@ public interface OffloadPolicies { Integer getManagedLedgerOffloadReadBufferSizeInBytes(); - void setOffloadersDirectory(String offloadersDirectory); + interface Builder { + + Builder offloadersDirectory(String offloadersDirectory); + + Builder managedLedgerOffloadDriver(String managedLedgerOffloadDriver); + + Builder managedLedgerOffloadMaxThreads(Integer managedLedgerOffloadMaxThreads); - void setManagedLedgerOffloadDriver(String managedLedgerOffloadDriver); + Builder managedLedgerOffloadPrefetchRounds(Integer managedLedgerOffloadPrefetchRounds); - void setManagedLedgerOffloadMaxThreads(Integer managedLedgerOffloadMaxThreads); + Builder managedLedgerOffloadThresholdInBytes(Long managedLedgerOffloadThresholdInBytes); - void setManagedLedgerOffloadPrefetchRounds(Integer managedLedgerOffloadPrefetchRounds); + Builder managedLedgerOffloadDeletionLagInMillis(Long managedLedgerOffloadDeletionLagInMillis); - void setManagedLedgerOffloadThresholdInBytes(Long managedLedgerOffloadThresholdInBytes); + Builder managedLedgerOffloadedReadPriority(OffloadedReadPriority managedLedgerOffloadedReadPriority); - void setManagedLedgerOffloadDeletionLagInMillis(Long managedLedgerOffloadDeletionLagInMillis); + Builder s3ManagedLedgerOffloadRegion(String s3ManagedLedgerOffloadRegion); - void setManagedLedgerOffloadedReadPriority(OffloadedReadPriority managedLedgerOffloadedReadPriority); + Builder s3ManagedLedgerOffloadBucket(String s3ManagedLedgerOffloadBucket); - void setS3ManagedLedgerOffloadRegion(String s3ManagedLedgerOffloadRegion); + Builder s3ManagedLedgerOffloadServiceEndpoint(String s3ManagedLedgerOffloadServiceEndpoint); - void setS3ManagedLedgerOffloadBucket(String s3ManagedLedgerOffloadBucket); + Builder s3ManagedLedgerOffloadMaxBlockSizeInBytes(Integer s3ManagedLedgerOffloadMaxBlockSizeInBytes); - void setS3ManagedLedgerOffloadServiceEndpoint(String s3ManagedLedgerOffloadServiceEndpoint); + Builder s3ManagedLedgerOffloadReadBufferSizeInBytes(Integer s3ManagedLedgerOffloadReadBufferSizeInBytes); - void setS3ManagedLedgerOffloadMaxBlockSizeInBytes(Integer s3ManagedLedgerOffloadMaxBlockSizeInBytes); + Builder s3ManagedLedgerOffloadCredentialId(String s3ManagedLedgerOffloadCredentialId); - void setS3ManagedLedgerOffloadReadBufferSizeInBytes(Integer s3ManagedLedgerOffloadReadBufferSizeInBytes); + Builder s3ManagedLedgerOffloadCredentialSecret(String s3ManagedLedgerOffloadCredentialSecret); - void setS3ManagedLedgerOffloadCredentialId(String s3ManagedLedgerOffloadCredentialId); + Builder s3ManagedLedgerOffloadRole(String s3ManagedLedgerOffloadRole); - void setS3ManagedLedgerOffloadCredentialSecret(String s3ManagedLedgerOffloadCredentialSecret); + Builder setS3ManagedLedgerOffloadRoleSessionName(String s3ManagedLedgerOffloadRoleSessionName); - void setS3ManagedLedgerOffloadRole(String s3ManagedLedgerOffloadRole); + Builder gcsManagedLedgerOffloadRegion(String gcsManagedLedgerOffloadRegion); - void setS3ManagedLedgerOffloadRoleSessionName(String s3ManagedLedgerOffloadRoleSessionName); + Builder gcsManagedLedgerOffloadBucket(String gcsManagedLedgerOffloadBucket); - void setGcsManagedLedgerOffloadRegion(String gcsManagedLedgerOffloadRegion); + Builder gcsManagedLedgerOffloadMaxBlockSizeInBytes(Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes); - void setGcsManagedLedgerOffloadBucket(String gcsManagedLedgerOffloadBucket); + Builder gcsManagedLedgerOffloadReadBufferSizeInBytes(Integer gcsManagedLedgerOffloadReadBufferSizeInBytes); - void setGcsManagedLedgerOffloadMaxBlockSizeInBytes(Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes); + Builder gcsManagedLedgerOffloadServiceAccountKeyFile(String gcsManagedLedgerOffloadServiceAccountKeyFile); - void setGcsManagedLedgerOffloadReadBufferSizeInBytes(Integer gcsManagedLedgerOffloadReadBufferSizeInBytes); + Builder fileSystemProfilePath(String fileSystemProfilePath); - void setGcsManagedLedgerOffloadServiceAccountKeyFile(String gcsManagedLedgerOffloadServiceAccountKeyFile); + Builder fileSystemURI(String fileSystemURI); - void setFileSystemProfilePath(String fileSystemProfilePath); + Builder managedLedgerOffloadBucket(String managedLedgerOffloadBucket); - void setFileSystemURI(String fileSystemURI); + Builder managedLedgerOffloadRegion(String managedLedgerOffloadRegion); - void setManagedLedgerOffloadBucket(String managedLedgerOffloadBucket); + Builder managedLedgerOffloadServiceEndpoint(String managedLedgerOffloadServiceEndpoint); - void setManagedLedgerOffloadRegion(String managedLedgerOffloadRegion); + Builder managedLedgerOffloadMaxBlockSizeInBytes(Integer managedLedgerOffloadMaxBlockSizeInBytes); - void setManagedLedgerOffloadServiceEndpoint(String managedLedgerOffloadServiceEndpoint); + Builder managedLedgerOffloadReadBufferSizeInBytes(Integer managedLedgerOffloadReadBufferSizeInBytes); - void setManagedLedgerOffloadMaxBlockSizeInBytes(Integer managedLedgerOffloadMaxBlockSizeInBytes); + OffloadPolicies build(); + } - void setManagedLedgerOffloadReadBufferSizeInBytes(Integer managedLedgerOffloadReadBufferSizeInBytes); + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.common.policies.data.OffloadPoliciesImpl"); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PartitionedTopicStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PartitionedTopicStats.java index 699fe6e470315..76ed0e366b4c9 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PartitionedTopicStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PartitionedTopicStats.java @@ -18,35 +18,15 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.HashMap; import java.util.Map; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; /** * Statistics for a partitioned topic. */ -public class PartitionedTopicStats extends TopicStats { +public interface PartitionedTopicStats extends TopicStats { - public PartitionedTopicMetadata metadata; - - public Map partitions; - - public PartitionedTopicStats() { - super(); - metadata = new PartitionedTopicMetadata(); - partitions = new HashMap<>(); - } - - public PartitionedTopicStats(PartitionedTopicMetadata metadata) { - this(); - this.metadata = metadata; - } - - @Override - public void reset() { - super.reset(); - partitions.clear(); - metadata.partitions = 0; - } + PartitionedTopicMetadata getMetadata(); + Map getPartitions(); } \ No newline at end of file diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index 86c0b2b8a19b5..b2376f82afe9a 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -24,6 +24,7 @@ import java.util.Objects; import java.util.Set; import lombok.ToString; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; /** * Definition of Pulsar policies. @@ -32,17 +33,17 @@ public class Policies { @SuppressWarnings("checkstyle:MemberName") - public final AuthPolicies auth_policies = new AuthPolicies(); + public final AuthPolicies auth_policies = AuthPolicies.builder().build(); @SuppressWarnings("checkstyle:MemberName") public Set replication_clusters = new HashSet<>(); public BundlesData bundles; @SuppressWarnings("checkstyle:MemberName") public Map backlog_quota_map = new HashMap<>(); @Deprecated - public Map clusterDispatchRate = new HashMap<>(); - public Map topicDispatchRate = new HashMap<>(); - public Map subscriptionDispatchRate = new HashMap<>(); - public Map replicatorDispatchRate = new HashMap<>(); + public Map clusterDispatchRate = new HashMap<>(); + public Map topicDispatchRate = new HashMap<>(); + public Map subscriptionDispatchRate = new HashMap<>(); + public Map replicatorDispatchRate = new HashMap<>(); public Map clusterSubscribeRate = new HashMap<>(); public PersistencePolicies persistence = null; diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PublisherStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PublisherStats.java index e3732fa553d07..b20c128f221bd 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PublisherStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PublisherStats.java @@ -24,119 +24,37 @@ /** * Statistics about a publisher. */ -public class PublisherStats { - private int count; +public interface PublisherStats { - public ProducerAccessMode accessMode; + ProducerAccessMode getAccessMode(); /** Total rate of messages published by this publisher (msg/s). */ - public double msgRateIn; + double getMsgRateIn(); /** Total throughput of messages published by this publisher (byte/s). */ - public double msgThroughputIn; + double getMsgThroughputIn(); /** Average message size published by this publisher. */ - public double averageMsgSize; + double getAverageMsgSize(); /** total chunked message count received. **/ - public double chunkedMessageRate; + double getChunkedMessageRate(); /** Id of this publisher. */ - public long producerId; + long getProducerId(); /** Producer name. */ - private int producerNameOffset = -1; - private int producerNameLength; + String getProducerName(); /** Address of this publisher. */ - private int addressOffset = -1; - private int addressLength; + String getAddress(); /** Timestamp of connection. */ - private int connectedSinceOffset = -1; - private int connectedSinceLength; + String getConnectedSince(); /** Client library version. */ - private int clientVersionOffset = -1; - private int clientVersionLength; - - /** - * In order to prevent multiple string objects under stats: create a string-buffer that stores data for all string - * place-holders. - */ - private StringBuilder stringBuffer = new StringBuilder(); + String getClientVersion(); /** Metadata (key/value strings) associated with this publisher. */ - public Map metadata; - - public PublisherStats add(PublisherStats stats) { - if (stats == null) { - throw new NullPointerException(); - } - this.count++; - this.msgRateIn += stats.msgRateIn; - this.msgThroughputIn += stats.msgThroughputIn; - double newAverageMsgSize = (this.averageMsgSize * (this.count - 1) + stats.averageMsgSize) / this.count; - this.averageMsgSize = newAverageMsgSize; - return this; - } - - public String getProducerName() { - return producerNameOffset == -1 ? null - : stringBuffer.substring(producerNameOffset, producerNameOffset + producerNameLength); - } - - public void setProducerName(String producerName) { - if (producerName == null) { - this.producerNameOffset = -1; - return; - } - this.producerNameOffset = this.stringBuffer.length(); - this.producerNameLength = producerName.length(); - this.stringBuffer.append(producerName); - } - - public String getAddress() { - return addressOffset == -1 ? null : stringBuffer.substring(addressOffset, addressOffset + addressLength); - } - - public void setAddress(String address) { - if (address == null) { - this.addressOffset = -1; - return; - } - this.addressOffset = this.stringBuffer.length(); - this.addressLength = address.length(); - this.stringBuffer.append(address); - } - - public String getConnectedSince() { - return connectedSinceOffset == -1 ? null - : stringBuffer.substring(connectedSinceOffset, connectedSinceOffset + connectedSinceLength); - } - - public void setConnectedSince(String connectedSince) { - if (connectedSince == null) { - this.connectedSinceOffset = -1; - return; - } - this.connectedSinceOffset = this.stringBuffer.length(); - this.connectedSinceLength = connectedSince.length(); - this.stringBuffer.append(connectedSince); - } - - public String getClientVersion() { - return clientVersionOffset == -1 ? null - : stringBuffer.substring(clientVersionOffset, clientVersionOffset + clientVersionLength); - } - - public void setClientVersion(String clientVersion) { - if (clientVersion == null) { - this.clientVersionOffset = -1; - return; - } - this.clientVersionOffset = this.stringBuffer.length(); - this.clientVersionLength = clientVersion.length(); - this.stringBuffer.append(clientVersion); - } + Map getMetadata(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java index 2969b512c6e52..06e881ba1c210 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java @@ -18,59 +18,44 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.Objects; - /** * Statistics about a replicator. */ -public class ReplicatorStats { +public interface ReplicatorStats { /** Total rate of messages received from the remote cluster (msg/s). */ - public double msgRateIn; + double getMsgRateIn(); /** Total throughput received from the remote cluster (bytes/s). */ - public double msgThroughputIn; + double getMsgThroughputIn(); /** Total rate of messages delivered to the replication-subscriber (msg/s). */ - public double msgRateOut; + double getMsgRateOut(); /** Total throughput delivered to the replication-subscriber (bytes/s). */ - public double msgThroughputOut; + double getMsgThroughputOut(); /** Total rate of messages expired (msg/s). */ - public double msgRateExpired; + double getMsgRateExpired(); /** Number of messages pending to be replicated to remote cluster. */ - public long replicationBacklog; + long getReplicationBacklog(); /** is the replication-subscriber up and running to replicate to remote cluster. */ - public boolean connected; + boolean isConnected(); /** Time in seconds from the time a message was produced to the time when it is about to be replicated. */ - public long replicationDelayInSeconds; + long getReplicationDelayInSeconds(); /** Address of incoming replication connection. */ - public String inboundConnection; + String getInboundConnection(); /** Timestamp of incoming connection establishment time. */ - public String inboundConnectedSince; + String getInboundConnectedSince(); /** Address of outbound replication connection. */ - public String outboundConnection; + String getOutboundConnection(); /** Timestamp of outbound connection establishment time. */ - public String outboundConnectedSince; - - public ReplicatorStats add(ReplicatorStats stats) { - Objects.requireNonNull(stats); - this.msgRateIn += stats.msgRateIn; - this.msgThroughputIn += stats.msgThroughputIn; - this.msgRateOut += stats.msgRateOut; - this.msgThroughputOut += stats.msgThroughputOut; - this.msgRateExpired += stats.msgRateExpired; - this.replicationBacklog += stats.replicationBacklog; - this.connected &= stats.connected; - this.replicationDelayInSeconds = Math.max(this.replicationDelayInSeconds, stats.replicationDelayInSeconds); - return this; - } + String getOutboundConnectedSince(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index 7685c72c24813..c9427fb54c00c 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -18,160 +18,91 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.ArrayList; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; /** * Statistics about subscription. */ -public class SubscriptionStats { +public interface SubscriptionStats { /** Total rate of messages delivered on this subscription (msg/s). */ - public double msgRateOut; + double getMsgRateOut(); /** Total throughput delivered on this subscription (bytes/s). */ - public double msgThroughputOut; + double getMsgThroughputOut(); /** Total bytes delivered to consumer (bytes). */ - public long bytesOutCounter; + long getBytesOutCounter(); /** Total messages delivered to consumer (msg). */ - public long msgOutCounter; + long getMsgOutCounter(); /** Total rate of messages redelivered on this subscription (msg/s). */ - public double msgRateRedeliver; - - /** - * Chunked message dispatch rate. - * @deprecated use {@link chunkedMessageRate)} - */ - @Deprecated - public int chuckedMessageRate; + double getMsgRateRedeliver(); /** Chunked message dispatch rate. */ - public int chunkedMessageRate; + int getChunkedMessageRate(); /** Number of messages in the subscription backlog. */ - public long msgBacklog; + long getMsgBacklog(); /** Size of backlog in byte. **/ - public long backlogSize; + long getBacklogSize(); /** Number of messages in the subscription backlog that do not contain the delay messages. */ - public long msgBacklogNoDelayed; + long getMsgBacklogNoDelayed(); /** Flag to verify if subscription is blocked due to reaching threshold of unacked messages. */ - public boolean blockedSubscriptionOnUnackedMsgs; + boolean isBlockedSubscriptionOnUnackedMsgs(); /** Number of delayed messages currently being tracked. */ - public long msgDelayed; + long getMsgDelayed(); /** Number of unacknowledged messages for the subscription. */ - public long unackedMessages; + long getUnackedMessages(); /** Whether this subscription is Exclusive or Shared or Failover. */ - public String type; + String getType(); /** The name of the consumer that is active for single active consumer subscriptions i.e. failover or exclusive. */ - public String activeConsumerName; + String getActiveConsumerName(); /** Total rate of messages expired on this subscription (msg/s). */ - public double msgRateExpired; + double getMsgRateExpired(); /** Total messages expired on this subscription. */ - public long totalMsgExpired; + long getTotalMsgExpired(); /** Last message expire execution timestamp. */ - public long lastExpireTimestamp; + long getLastExpireTimestamp(); /** Last received consume flow command timestamp. */ - public long lastConsumedFlowTimestamp; + long getLastConsumedFlowTimestamp(); /** Last consume message timestamp. */ - public long lastConsumedTimestamp; + long getLastConsumedTimestamp(); /** Last acked message timestamp. */ - public long lastAckedTimestamp; + long getLastAckedTimestamp(); /** Last MarkDelete position advanced timesetamp. */ - public long lastMarkDeleteAdvancedTimestamp; + long getLastMarkDeleteAdvancedTimestamp(); /** List of connected consumers on this subscription w/ their stats. */ - public List consumers; + List getConsumers(); /** Tells whether this subscription is durable or ephemeral (eg.: from a reader). */ - public boolean isDurable; + boolean isDurable(); /** Mark that the subscription state is kept in sync across different regions. */ - public boolean isReplicated; + boolean isReplicated(); /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - public Map consumersAfterMarkDeletePosition; + Map getConsumersAfterMarkDeletePosition(); /** The number of non-contiguous deleted messages ranges. */ - public int nonContiguousDeletedMessagesRanges; + int getNonContiguousDeletedMessagesRanges(); /** The serialized size of non-contiguous deleted messages ranges. */ - public int nonContiguousDeletedMessagesRangesSerializedSize; - - public SubscriptionStats() { - this.consumers = new ArrayList<>(); - this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); - } - - public void reset() { - msgRateOut = 0; - msgThroughputOut = 0; - bytesOutCounter = 0; - msgOutCounter = 0; - msgRateRedeliver = 0; - msgBacklog = 0; - backlogSize = 0; - msgBacklogNoDelayed = 0; - unackedMessages = 0; - msgRateExpired = 0; - totalMsgExpired = 0; - lastExpireTimestamp = 0L; - lastMarkDeleteAdvancedTimestamp = 0L; - consumers.clear(); - consumersAfterMarkDeletePosition.clear(); - nonContiguousDeletedMessagesRanges = 0; - nonContiguousDeletedMessagesRangesSerializedSize = 0; - } - - // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current - // stats - public SubscriptionStats add(SubscriptionStats stats) { - Objects.requireNonNull(stats); - this.msgRateOut += stats.msgRateOut; - this.msgThroughputOut += stats.msgThroughputOut; - this.bytesOutCounter += stats.bytesOutCounter; - this.msgOutCounter += stats.msgOutCounter; - this.msgRateRedeliver += stats.msgRateRedeliver; - this.msgBacklog += stats.msgBacklog; - this.backlogSize += stats.backlogSize; - this.msgBacklogNoDelayed += stats.msgBacklogNoDelayed; - this.msgDelayed += stats.msgDelayed; - this.unackedMessages += stats.unackedMessages; - this.msgRateExpired += stats.msgRateExpired; - this.totalMsgExpired += stats.totalMsgExpired; - this.isReplicated |= stats.isReplicated; - this.isDurable |= stats.isDurable; - if (this.consumers.size() != stats.consumers.size()) { - for (int i = 0; i < stats.consumers.size(); i++) { - ConsumerStats consumerStats = new ConsumerStats(); - this.consumers.add(consumerStats.add(stats.consumers.get(i))); - } - } else { - for (int i = 0; i < stats.consumers.size(); i++) { - this.consumers.get(i).add(stats.consumers.get(i)); - } - } - this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); - this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; - this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; - return this; - } + int getNonContiguousDeletedMessagesRangesSerializedSize(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TenantInfo.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TenantInfo.java index dc8625e0cb5cd..f1f6658f0a13a 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TenantInfo.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TenantInfo.java @@ -19,13 +19,20 @@ package org.apache.pulsar.common.policies.data; import java.util.Set; +import org.apache.pulsar.client.admin.utils.ReflectionUtils; public interface TenantInfo { Set getAdminRoles(); - void setAdminRoles(Set adminRoles); - Set getAllowedClusters(); - void setAllowedClusters(Set allowedClusters); + interface Builder { + Builder adminRoles(Set adminRoles); + Builder allowedClusters(Set allowedClusters); + TenantInfo build(); + } + + static Builder builder() { + return ReflectionUtils.newBuilder("org.apache.pulsar.common.policies.data.TenantInfoImpl"); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java index a86b928187697..afc18108686dc 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java @@ -18,172 +18,71 @@ */ package org.apache.pulsar.common.policies.data; -import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.TreeMap; /** * Statistics for a Pulsar topic. */ -public class TopicStats { - private int count; - +public interface TopicStats { /** Total rate of messages published on the topic (msg/s). */ - public double msgRateIn; + double getMsgRateIn(); /** Total throughput of messages published on the topic (byte/s). */ - public double msgThroughputIn; + double getMsgThroughputIn(); /** Total rate of messages dispatched for the topic (msg/s). */ - public double msgRateOut; + double getMsgRateOut(); /** Total throughput of messages dispatched for the topic (byte/s). */ - public double msgThroughputOut; + double getMsgThroughputOut(); /** Total bytes published to the topic (bytes). */ - public long bytesInCounter; + long getBytesInCounter(); /** Total messages published to the topic (msg). */ - public long msgInCounter; + long getMsgInCounter(); /** Total bytes delivered to consumer (bytes). */ - public long bytesOutCounter; + long getBytesOutCounter(); /** Total messages delivered to consumer (msg). */ - public long msgOutCounter; + long getMsgOutCounter(); /** Average size of published messages (bytes). */ - public double averageMsgSize; + double getAverageMsgSize(); /** Topic has chunked message published on it. */ - public boolean msgChunkPublished; + boolean isMsgChunkPublished(); /** Space used to store the messages for the topic (bytes). */ - public long storageSize; + long getStorageSize(); /** Get estimated total unconsumed or backlog size in bytes. */ - public long backlogSize; + long getBacklogSize(); /** Space used to store the offloaded messages for the topic/. */ - public long offloadedStorageSize; + long getOffloadedStorageSize(); /** List of connected publishers on this topic w/ their stats. */ - public List publishers; + List getPublishers(); - public int waitingPublishers; + int getWaitingPublishers(); /** Map of subscriptions with their individual statistics. */ - public Map subscriptions; + Map getSubscriptions(); /** Map of replication statistics by remote cluster context. */ - public Map replication; + Map getReplication(); - public String deduplicationStatus; + String getDeduplicationStatus(); /** The topic epoch or empty if not set. */ - public Long topicEpoch; + Long getTopicEpoch(); /** The number of non-contiguous deleted messages ranges. */ - public int nonContiguousDeletedMessagesRanges; + int getNonContiguousDeletedMessagesRanges(); /** The serialized size of non-contiguous deleted messages ranges. */ - public int nonContiguousDeletedMessagesRangesSerializedSize; - - public TopicStats() { - this.publishers = new ArrayList<>(); - this.subscriptions = new HashMap<>(); - this.replication = new TreeMap<>(); - } - - public void reset() { - this.count = 0; - this.msgRateIn = 0; - this.msgThroughputIn = 0; - this.msgRateOut = 0; - this.msgThroughputOut = 0; - this.averageMsgSize = 0; - this.storageSize = 0; - this.backlogSize = 0; - this.bytesInCounter = 0; - this.msgInCounter = 0; - this.bytesOutCounter = 0; - this.msgOutCounter = 0; - this.publishers.clear(); - this.subscriptions.clear(); - this.waitingPublishers = 0; - this.replication.clear(); - this.deduplicationStatus = null; - this.topicEpoch = null; - this.nonContiguousDeletedMessagesRanges = 0; - this.nonContiguousDeletedMessagesRangesSerializedSize = 0; - this.offloadedStorageSize = 0; - } - - // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current - // stats. - public TopicStats add(TopicStats stats) { - Objects.requireNonNull(stats); - this.count++; - this.msgRateIn += stats.msgRateIn; - this.msgThroughputIn += stats.msgThroughputIn; - this.msgRateOut += stats.msgRateOut; - this.msgThroughputOut += stats.msgThroughputOut; - this.bytesInCounter += stats.bytesInCounter; - this.msgInCounter += stats.msgInCounter; - this.bytesOutCounter += stats.bytesOutCounter; - this.msgOutCounter += stats.msgOutCounter; - this.waitingPublishers += stats.waitingPublishers; - double newAverageMsgSize = (this.averageMsgSize * (this.count - 1) + stats.averageMsgSize) / this.count; - this.averageMsgSize = newAverageMsgSize; - this.storageSize += stats.storageSize; - this.backlogSize += stats.backlogSize; - this.offloadedStorageSize += stats.offloadedStorageSize; - this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; - this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; - if (this.publishers.size() != stats.publishers.size()) { - for (int i = 0; i < stats.publishers.size(); i++) { - PublisherStats publisherStats = new PublisherStats(); - this.publishers.add(publisherStats.add(stats.publishers.get(i))); - } - } else { - for (int i = 0; i < stats.publishers.size(); i++) { - this.publishers.get(i).add(stats.publishers.get(i)); - } - } - if (this.subscriptions.size() != stats.subscriptions.size()) { - for (String subscription : stats.subscriptions.keySet()) { - SubscriptionStats subscriptionStats = new SubscriptionStats(); - this.subscriptions.put(subscription, subscriptionStats.add(stats.subscriptions.get(subscription))); - } - } else { - for (String subscription : stats.subscriptions.keySet()) { - if (this.subscriptions.get(subscription) != null) { - this.subscriptions.get(subscription).add(stats.subscriptions.get(subscription)); - } else { - SubscriptionStats subscriptionStats = new SubscriptionStats(); - this.subscriptions.put(subscription, subscriptionStats.add(stats.subscriptions.get(subscription))); - } - } - } - if (this.replication.size() != stats.replication.size()) { - for (String repl : stats.replication.keySet()) { - ReplicatorStats replStats = new ReplicatorStats(); - this.replication.put(repl, replStats.add(stats.replication.get(repl))); - } - } else { - for (String repl : stats.replication.keySet()) { - if (this.replication.get(repl) != null) { - this.replication.get(repl).add(stats.replication.get(repl)); - } else { - ReplicatorStats replStats = new ReplicatorStats(); - this.replication.put(repl, replStats.add(stats.replication.get(repl))); - } - } - } - return this; - } - + int getNonContiguousDeletedMessagesRangesSerializedSize(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoSubscriptionCreationOverrideImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoSubscriptionCreationOverrideImpl.java new file mode 100644 index 0000000000000..ac1af07d044a1 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoSubscriptionCreationOverrideImpl.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; + +/** + * Override of autoSubscriptionCreation settings on a namespace level. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public class AutoSubscriptionCreationOverrideImpl implements AutoSubscriptionCreationOverride { + private boolean allowAutoSubscriptionCreation; + + public static AutoSubscriptionCreationOverrideImplBuilder builder() { + return new AutoSubscriptionCreationOverrideImplBuilder(); + } + + public static class AutoSubscriptionCreationOverrideImplBuilder + implements AutoSubscriptionCreationOverride.Builder { + private boolean allowAutoSubscriptionCreation; + + public AutoSubscriptionCreationOverrideImplBuilder allowAutoSubscriptionCreation( + boolean allowAutoSubscriptionCreation) { + this.allowAutoSubscriptionCreation = allowAutoSubscriptionCreation; + return this; + } + + public AutoSubscriptionCreationOverrideImpl build() { + return new AutoSubscriptionCreationOverrideImpl(allowAutoSubscriptionCreation); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoTopicCreationOverrideImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoTopicCreationOverrideImpl.java new file mode 100644 index 0000000000000..1ce60d1c78b30 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/AutoTopicCreationOverrideImpl.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.TopicType; + +/** + * Override of autoTopicCreation settings on a namespace level. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class AutoTopicCreationOverrideImpl implements AutoTopicCreationOverride { + private boolean allowAutoTopicCreation; + private String topicType; + private Integer defaultNumPartitions; + + public static boolean isValidOverride(AutoTopicCreationOverride override) { + if (override == null) { + return false; + } + if (override.isAllowAutoTopicCreation()) { + if (!TopicType.isValidTopicType(override.getTopicType())) { + return false; + } + if (TopicType.PARTITIONED.toString().equals(override.getTopicType())) { + if (override.getDefaultNumPartitions() == null) { + return false; + } + if (!(override.getDefaultNumPartitions() > 0)) { + return false; + } + } else if (TopicType.NON_PARTITIONED.toString().equals(override.getTopicType())) { + if (override.getDefaultNumPartitions() != null) { + return false; + } + } + } + return true; + } + + public static AutoTopicCreationOverrideImplBuilder builder() { + return new AutoTopicCreationOverrideImplBuilder(); + } + + public static class AutoTopicCreationOverrideImplBuilder implements AutoTopicCreationOverride.Builder { + private boolean allowAutoTopicCreation; + private String topicType; + private Integer defaultNumPartitions; + + public AutoTopicCreationOverrideImplBuilder allowAutoTopicCreation(boolean allowAutoTopicCreation) { + this.allowAutoTopicCreation = allowAutoTopicCreation; + return this; + } + + public AutoTopicCreationOverrideImplBuilder topicType(String topicType) { + this.topicType = topicType; + return this; + } + + public AutoTopicCreationOverrideImplBuilder defaultNumPartitions(Integer defaultNumPartitions) { + this.defaultNumPartitions = defaultNumPartitions; + return this; + } + + public AutoTopicCreationOverrideImpl build() { + return new AutoTopicCreationOverrideImpl(allowAutoTopicCreation, topicType, defaultNumPartitions); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BacklogQuotaImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BacklogQuotaImpl.java new file mode 100644 index 0000000000000..f60f4ca10de6a --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BacklogQuotaImpl.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BacklogQuota; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class BacklogQuotaImpl implements BacklogQuota { + private long limitSize; + // backlog quota by time in second + private int limitTime; + private RetentionPolicy policy; + + public static BacklogQuotaImplBuilder builder() { + return new BacklogQuotaImplBuilder(); + } + + public static class BacklogQuotaImplBuilder implements BacklogQuota.Builder { + private long limitSize; + private int limitTime = -1; + private RetentionPolicy retentionPolicy; + + public BacklogQuotaImplBuilder limitSize(long limitSize) { + this.limitSize = limitSize; + return this; + } + + public BacklogQuotaImplBuilder limitTime(int limitTime) { + this.limitTime = limitTime; + return this; + } + + public BacklogQuotaImplBuilder retentionPolicy(RetentionPolicy retentionPolicy) { + this.retentionPolicy = retentionPolicy; + return this; + } + + public BacklogQuotaImpl build() { + return new BacklogQuotaImpl(limitSize, limitTime, retentionPolicy); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieAffinityGroupDataImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieAffinityGroupDataImpl.java new file mode 100644 index 0000000000000..017afe4506217 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieAffinityGroupDataImpl.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; + +/** + * Description of a BookKeeper's affinity group. + */ +@Data +@NoArgsConstructor +@AllArgsConstructor +public final class BookieAffinityGroupDataImpl implements BookieAffinityGroupData { + + private String bookkeeperAffinityGroupPrimary; + private String bookkeeperAffinityGroupSecondary; + + public static BookieAffinityGroupDataImplBuilder builder() { + return new BookieAffinityGroupDataImplBuilder(); + } + + public static class BookieAffinityGroupDataImplBuilder implements BookieAffinityGroupData.Builder { + private String bookkeeperAffinityGroupPrimary; + private String bookkeeperAffinityGroupSecondary; + + public BookieAffinityGroupDataImplBuilder bookkeeperAffinityGroupPrimary( + String bookkeeperAffinityGroupPrimary) { + this.bookkeeperAffinityGroupPrimary = bookkeeperAffinityGroupPrimary; + return this; + } + + public BookieAffinityGroupDataImplBuilder bookkeeperAffinityGroupSecondary( + String bookkeeperAffinityGroupSecondary) { + this.bookkeeperAffinityGroupSecondary = bookkeeperAffinityGroupSecondary; + return this; + } + + public BookieAffinityGroupDataImpl build() { + return new BookieAffinityGroupDataImpl(bookkeeperAffinityGroupPrimary, bookkeeperAffinityGroupSecondary); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieInfoImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieInfoImpl.java new file mode 100644 index 0000000000000..de316e612f3c1 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookieInfoImpl.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.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BookieInfo; + +/** + * Bookie information. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class BookieInfoImpl implements BookieInfo { + private String rack; + private String hostname; + + public static BookieInfoImplBuilder builder() { + return new BookieInfoImplBuilder(); + } + + public static class BookieInfoImplBuilder implements BookieInfo.Builder { + private String rack; + private String hostname; + + public BookieInfoImplBuilder rack(String rack) { + this.rack = rack; + return this; + } + + public BookieInfoImplBuilder hostname(String hostname) { + this.hostname = hostname; + return this; + } + + public BookieInfoImpl build() { + return new BookieInfoImpl(rack, hostname); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookiesClusterInfoImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookiesClusterInfoImpl.java new file mode 100644 index 0000000000000..0e983d3b35027 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BookiesClusterInfoImpl.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BookiesClusterInfo; +import org.apache.pulsar.common.policies.data.RawBookieInfo; + +/** + * Raw bookies information. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class BookiesClusterInfoImpl implements BookiesClusterInfo { + + private List bookies; + + public static BookiesClusterInfoImplBuilder builder() { + return new BookiesClusterInfoImplBuilder(); + } + + public static class BookiesClusterInfoImplBuilder implements BookiesClusterInfo.Builder{ + private List bookies; + + public BookiesClusterInfoImplBuilder bookies(List bookies) { + this.bookies = bookies; + return this; + } + + public BookiesClusterInfoImpl build() { + return new BookiesClusterInfoImpl(bookies); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerInfoImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerInfoImpl.java new file mode 100644 index 0000000000000..43bb05fd41f97 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerInfoImpl.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BrokerInfo; + +/** + * Broker Information. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class BrokerInfoImpl implements BrokerInfo { + private String serviceUrl; + + public static BrokerInfoImplBuilder builder() { + return new BrokerInfoImplBuilder(); + } + + public static class BrokerInfoImplBuilder implements BrokerInfo.Builder { + private String serviceUrl; + + public BrokerInfoImplBuilder serviceUrl(String serviceUrl) { + this.serviceUrl = serviceUrl; + return this; + } + + public BrokerInfoImpl build() { + return new BrokerInfoImpl(serviceUrl); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerStatusImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerStatusImpl.java new file mode 100644 index 0000000000000..a2cb6ac357b41 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BrokerStatusImpl.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BrokerStatus; + +/** + * Information about the broker status. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class BrokerStatusImpl implements BrokerStatus { + private String brokerAddress; + private boolean active; + private int loadFactor; + + public static BrokerStatusImplBuilder builder() { + return new BrokerStatusImplBuilder(); + } + + @Override + public int compareTo(BrokerStatus other) { + int result = Integer.compare(this.loadFactor, other.getLoadFactor()); + if (result == 0) { + result = this.brokerAddress.compareTo(other.getBrokerAddress()); + } + return result; + } + + public static class BrokerStatusImplBuilder implements BrokerStatus.Builder { + private String brokerAddress; + private boolean active; + private int loadFactor; + + public BrokerStatusImplBuilder brokerAddress(String brokerAddress) { + this.brokerAddress = brokerAddress; + return this; + } + + public BrokerStatusImplBuilder active(boolean active) { + this.active = active; + return this; + } + + public BrokerStatusImplBuilder loadFactor(int loadFactor) { + this.loadFactor = loadFactor; + return this; + } + + public BrokerStatusImpl build() { + return new BrokerStatusImpl(brokerAddress, active, loadFactor); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BundlesDataImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BundlesDataImpl.java new file mode 100644 index 0000000000000..92e8900a9cb2e --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/BundlesDataImpl.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.BundlesData; + +/** + * Holder for bundles. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class BundlesDataImpl implements BundlesData { + private List boundaries; + private int numBundles; + + public static BundlesDataImplBuilder builder() { + return new BundlesDataImplBuilder(); + } + + public static class BundlesDataImplBuilder implements BundlesData.Builder { + private List boundaries; + private int numBundles = 0; + + public BundlesDataImplBuilder boundaries(List boundaries) { + this.boundaries = boundaries; + return this; + } + + public BundlesDataImplBuilder numBundles(int numBundles) { + this.numBundles = numBundles; + return this; + } + + public BundlesDataImpl build() { + return new BundlesDataImpl(boundaries, numBundles); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DelayedDeliveryPoliciesImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DelayedDeliveryPoliciesImpl.java new file mode 100644 index 0000000000000..45ab10f6500d8 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DelayedDeliveryPoliciesImpl.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.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; + +/** + * Definition of the delayed delivery policy. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class DelayedDeliveryPoliciesImpl implements DelayedDeliveryPolicies { + private long tickTime; + private boolean active; + + public static DelayedDeliveryPoliciesImplBuilder builder() { + return new DelayedDeliveryPoliciesImplBuilder(); + } + + public static class DelayedDeliveryPoliciesImplBuilder implements DelayedDeliveryPolicies.Builder { + private long tickTime; + private boolean active; + + public DelayedDeliveryPoliciesImplBuilder tickTime(long tickTime) { + this.tickTime = tickTime; + return this; + } + + public DelayedDeliveryPoliciesImplBuilder active(boolean active) { + this.active = active; + return this; + } + + public DelayedDeliveryPoliciesImpl build() { + return new DelayedDeliveryPoliciesImpl(tickTime, active); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DispatchRateImpl.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DispatchRateImpl.java new file mode 100644 index 0000000000000..4de7f04463d5e --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/DispatchRateImpl.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.apache.pulsar.common.policies.data.DispatchRate; + +/** + * Dispatch rate. + */ +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class DispatchRateImpl implements DispatchRate { + + private int dispatchThrottlingRateInMsg; + private long dispatchThrottlingRateInByte; + private boolean relativeToPublishRate; + private int ratePeriodInSecond; + + public static DispatchRateImplBuilder builder() { + return new DispatchRateImplBuilder(); + } + + public static class DispatchRateImplBuilder implements DispatchRate.Builder { + + private int dispatchThrottlingRateInMsg = -1; + private long dispatchThrottlingRateInByte = -1; + private boolean relativeToPublishRate = false; /* throttles dispatch relatively publish-rate */ + private int ratePeriodInSecond = 1; /* by default dispatch-rate will be calculate per 1 second */ + + + public DispatchRateImplBuilder dispatchThrottlingRateInMsg(int dispatchThrottlingRateInMsg) { + this.dispatchThrottlingRateInMsg = dispatchThrottlingRateInMsg; + return this; + } + + public DispatchRateImplBuilder dispatchThrottlingRateInByte(long dispatchThrottlingRateInByte) { + this.dispatchThrottlingRateInByte = dispatchThrottlingRateInByte; + return this; + } + + public DispatchRateImplBuilder relativeToPublishRate(boolean relativeToPublishRate) { + this.relativeToPublishRate = relativeToPublishRate; + return this; + } + + public DispatchRateImplBuilder ratePeriodInSecond(int ratePeriodInSecond) { + this.ratePeriodInSecond = ratePeriodInSecond; + return this; + } + + public DispatchRateImpl build() { + return new DispatchRateImpl(dispatchThrottlingRateInMsg, dispatchThrottlingRateInByte, + relativeToPublishRate, ratePeriodInSecond); + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/package-info.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/package-info.java new file mode 100644 index 0000000000000..7567a1d390b76 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/impl/package-info.java @@ -0,0 +1,19 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.impl; \ No newline at end of file diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactory.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactory.java index 6fa45aa20505d..267eea147d6bb 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactory.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactory.java @@ -29,9 +29,9 @@ public class AutoFailoverPolicyFactory { public static AutoFailoverPolicy create(AutoFailoverPolicyData policyData) { // TODO: Add more policy types when needed - if (!AutoFailoverPolicyType.min_available.equals(policyData.getPolicy_type())) { + if (!AutoFailoverPolicyType.min_available.equals(policyData.getPolicyType())) { // right now, only support one type of policy: MinAvailablePolicy - throw new IllegalArgumentException("Unrecognized auto_failover_policy: " + policyData.getPolicy_type()); + throw new IllegalArgumentException("Unrecognized auto_failover_policy: " + policyData.getPolicyType()); } return new MinAvailablePolicy(policyData); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/MinAvailablePolicy.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/MinAvailablePolicy.java index 99200ffd434b5..6d9cece4ae5f2 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/MinAvailablePolicy.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/impl/MinAvailablePolicy.java @@ -18,8 +18,9 @@ */ package org.apache.pulsar.common.policies.impl; -import java.util.Objects; import java.util.SortedSet; +import lombok.AllArgsConstructor; +import lombok.Data; import org.apache.pulsar.common.policies.AutoFailoverPolicy; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; @@ -28,6 +29,8 @@ /** * Implementation of min available policy. */ +@Data +@AllArgsConstructor public class MinAvailablePolicy extends AutoFailoverPolicy { private static final String MIN_LIMIT_KEY = "min_limit"; private static final String USAGE_THRESHOLD_KEY = "usage_threshold"; @@ -38,13 +41,8 @@ public class MinAvailablePolicy extends AutoFailoverPolicy { @SuppressWarnings("checkstyle:MemberName") public int usage_threshold; - MinAvailablePolicy(int minLimit, int usageThreshold) { - this.min_limit = minLimit; - this.usage_threshold = usageThreshold; - } - public MinAvailablePolicy(AutoFailoverPolicyData policyData) { - if (!policyData.getPolicy_type().equals(AutoFailoverPolicyType.min_available)) { + if (!policyData.getPolicyType().equals(AutoFailoverPolicyType.min_available)) { throw new IllegalArgumentException(); } if (!policyData.getParameters().containsKey(MIN_LIMIT_KEY)) { @@ -57,20 +55,6 @@ public MinAvailablePolicy(AutoFailoverPolicyData policyData) { this.usage_threshold = Integer.parseInt(policyData.getParameters().get(USAGE_THRESHOLD_KEY)); } - @Override - public int hashCode() { - return Objects.hash(min_limit, usage_threshold); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof MinAvailablePolicy) { - MinAvailablePolicy other = (MinAvailablePolicy) obj; - return Objects.equals(min_limit, other.min_limit) && Objects.equals(usage_threshold, other.usage_threshold); - } - return false; - } - @Override public boolean isBrokerAvailable(BrokerStatus brokerStatus) { return brokerStatus.isActive() @@ -93,9 +77,4 @@ public boolean shouldFailoverToSecondary(SortedSet primaryCandidat public boolean shouldFailoverToSecondary(int totalPrimaryCandidates) { return totalPrimaryCandidates < this.min_limit; } - - @Override - public String toString() { - return String.format("[policy_type=min_available min_limit=%s usage_threshold=%s]", min_limit, usage_threshold); - } } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index f5f9e4994fcac..ccf7890559b80 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -270,12 +270,12 @@ public CompletableFuture createNamespaceAsync(String namespace, Set createNamespaceAsync(String namespace, int numBundles) { - return createNamespaceAsync(namespace, new BundlesData(numBundles)); + return createNamespaceAsync(namespace, BundlesData.builder().numBundles(numBundles).build()); } @Override diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java index 6ad108890e3c7..953fb98bee530 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java @@ -126,7 +126,7 @@ public void createTenant(String tenant, TenantInfo config) throws PulsarAdminExc @Override public CompletableFuture createTenantAsync(String tenant, TenantInfo config) { WebTarget path = adminTenants.path(tenant); - return asyncPutRequest(path, Entity.entity((TenantInfoImpl) config, MediaType.APPLICATION_JSON)); + return asyncPutRequest(path, Entity.entity(config, MediaType.APPLICATION_JSON)); } @Override diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index c38bbc0eefcd4..267a2a0969883 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -797,7 +797,7 @@ public CompletableFuture getPartitionedStatsAsync(String @Override public void completed(PartitionedTopicStats response) { if (!perPartition) { - response.partitions.clear(); + response.getPartitions().clear(); } future.complete(response); } diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index 180c8fe6627c6..0d19d9ae8d4e1 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -77,6 +77,7 @@ import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; @@ -190,10 +191,10 @@ public void clusters() throws Exception { verify(mockClusters).getCluster("use"); clusters.run(split("create use --url http://my-service.url:8080")); - verify(mockClusters).createCluster("use", new ClusterDataImpl("http://my-service.url:8080", null)); + verify(mockClusters).createCluster("use", ClusterData.builder().serviceUrl("http://my-service.url:8080").build()); clusters.run(split("update use --url http://my-service.url:8080")); - verify(mockClusters).updateCluster("use", new ClusterDataImpl("http://my-service.url:8080", null)); + verify(mockClusters).updateCluster("use", ClusterData.builder().serviceUrl("http://my-service.url:8080").build()); clusters.run(split("delete use")); verify(mockClusters).deleteCluster("use"); @@ -205,8 +206,9 @@ public void clusters() throws Exception { verify(mockClusters).getFailureDomain("use", "domain"); clusters.run(split("create-failure-domain use --domain-name domain --broker-list b1")); - FailureDomainImpl domain = new FailureDomainImpl(); - domain.setBrokers(Sets.newHashSet("b1")); + FailureDomain domain = FailureDomain.builder() + .brokers(Collections.singleton("b1")) + .build(); verify(mockClusters).createFailureDomain("use", "domain", domain); clusters.run(split("update-failure-domain use --domain-name domain --broker-list b1")); @@ -223,12 +225,18 @@ public void clusters() throws Exception { clusters.run( split("create my-cluster --url http://my-service.url:8080 --url-secure https://my-service.url:4443")); verify(mockClusters).createCluster("my-cluster", - new ClusterDataImpl("http://my-service.url:8080", "https://my-service.url:4443")); + ClusterData.builder() + .serviceUrl("http://my-service.url:8080") + .serviceUrlTls("https://my-service.url:4443") + .build()); clusters.run( split("update my-cluster --url http://my-service.url:8080 --url-secure https://my-service.url:4443")); verify(mockClusters).updateCluster("my-cluster", - new ClusterDataImpl("http://my-service.url:8080", "https://my-service.url:4443")); + ClusterData.builder() + .serviceUrl("http://my-service.url:8080") + .serviceUrlTls("https://my-service.url:4443") + .build()); clusters.run(split("delete my-cluster")); verify(mockClusters).deleteCluster("my-cluster"); @@ -244,10 +252,16 @@ public void clusters() throws Exception { clusters = new CmdClusters(() -> admin); clusters.run(split("create my-secure-cluster --url-secure https://my-service.url:4443")); - verify(mockClusters).createCluster("my-secure-cluster", new ClusterDataImpl(null, "https://my-service.url:4443")); + verify(mockClusters).createCluster("my-secure-cluster", + ClusterData.builder() + .serviceUrlTls("https://my-service.url:4443") + .build()); clusters.run(split("update my-secure-cluster --url-secure https://my-service.url:4443")); - verify(mockClusters).updateCluster("my-secure-cluster", new ClusterDataImpl(null, "https://my-service.url:4443")); + verify(mockClusters).updateCluster("my-secure-cluster", + ClusterData.builder() + .serviceUrlTls("https://my-service.url:4443") + .build()); clusters.run(split("delete my-secure-cluster")); verify(mockClusters).deleteCluster("my-secure-cluster"); @@ -257,21 +271,22 @@ public void clusters() throws Exception { clusters.run(split("create my-tls-cluster --url-secure https://my-service.url:4443 --tls-enable " + "--tls-enable-keystore --tls-trust-store-type JKS --tls-trust-store /var/private/tls/client.truststore.jks " + "--tls-trust-store-pwd clientpw")); - ClusterData data = new ClusterDataImpl(null, "https://my-service.url:4443"); - data.setBrokerClientTlsEnabled(true) - .setBrokerClientTlsEnabledWithKeyStore(true) - .setBrokerClientTlsTrustStoreType("JKS") - .setBrokerClientTlsTrustStore("/var/private/tls/client.truststore.jks") - .setBrokerClientTlsTrustStorePassword("clientpw"); - verify(mockClusters).createCluster("my-tls-cluster", data); + ClusterData.Builder data = ClusterData.builder() + .serviceUrlTls("https://my-service.url:4443") + .brokerClientTlsEnabled(true) + .brokerClientTlsEnabledWithKeyStore(true) + .brokerClientTlsTrustStoreType("JKS") + .brokerClientTlsTrustStore("/var/private/tls/client.truststore.jks") + .brokerClientTlsTrustStorePassword("clientpw"); + verify(mockClusters).createCluster("my-tls-cluster", data.build()); clusters.run(split("update my-tls-cluster --url-secure https://my-service.url:4443 --tls-enable " + "--tls-trust-certs-filepath /path/to/ca.cert.pem")); - data.setBrokerClientTlsEnabledWithKeyStore(false) - .setBrokerClientTlsTrustStore(null) - .setBrokerClientTlsTrustStorePassword(null) - .setBrokerClientTrustCertsFilePath("/path/to/ca.cert.pem"); - verify(mockClusters).updateCluster("my-tls-cluster", data); + data.brokerClientTlsEnabledWithKeyStore(false) + .brokerClientTlsTrustStore(null) + .brokerClientTlsTrustStorePassword(null) + .brokerClientTrustCertsFilePath("/path/to/ca.cert.pem"); + verify(mockClusters).updateCluster("my-tls-cluster", data.build()); } @Test @@ -357,7 +372,10 @@ public void namespaces() throws Exception { namespaces .run(split("set-bookie-affinity-group myprop/clust/ns1 --primary-group test1 --secondary-group test2")); verify(mockNamespaces).setBookieAffinityGroup("myprop/clust/ns1", - new BookieAffinityGroupData("test1", "test2")); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary("test1") + .bookkeeperAffinityGroupSecondary("test2") + .build()); namespaces.run(split("get-bookie-affinity-group myprop/clust/ns1")); verify(mockNamespaces).getBookieAffinityGroup("myprop/clust/ns1"); @@ -366,7 +384,11 @@ public void namespaces() throws Exception { verify(mockNamespaces).deleteBookieAffinityGroup("myprop/clust/ns1"); namespaces.run(split("set-replicator-dispatch-rate myprop/clust/ns1 -md 10 -bd 11 -dt 12")); - verify(mockNamespaces).setReplicatorDispatchRate("myprop/clust/ns1", new DispatchRate(10, 11, 12)); + verify(mockNamespaces).setReplicatorDispatchRate("myprop/clust/ns1", DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(11) + .ratePeriodInSecond(12) + .build()); namespaces.run(split("get-replicator-dispatch-rate myprop/clust/ns1")); verify(mockNamespaces).getReplicatorDispatchRate("myprop/clust/ns1"); @@ -392,7 +414,10 @@ public void namespaces() throws Exception { namespaces.run(split("set-backlog-quota myprop/clust/ns1 -p producer_request_hold -l 10")); verify(mockNamespaces).setBacklogQuota("myprop/clust/ns1", - new BacklogQuota(10, RetentionPolicy.producer_request_hold)); + BacklogQuota.builder() + .limitSize(10) + .retentionPolicy(RetentionPolicy.producer_request_hold) + .build()); mockNamespaces = mock(Namespaces.class); when(admin.namespaces()).thenReturn(mockNamespaces); @@ -400,7 +425,10 @@ public void namespaces() throws Exception { namespaces.run(split("set-backlog-quota myprop/clust/ns1 -p producer_exception -l 10K")); verify(mockNamespaces).setBacklogQuota("myprop/clust/ns1", - new BacklogQuota(10 * 1024, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); mockNamespaces = mock(Namespaces.class); when(admin.namespaces()).thenReturn(mockNamespaces); @@ -408,7 +436,10 @@ public void namespaces() throws Exception { namespaces.run(split("set-backlog-quota myprop/clust/ns1 -p producer_exception -l 10M")); verify(mockNamespaces).setBacklogQuota("myprop/clust/ns1", - new BacklogQuota(10 * 1024 * 1024, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10 * 1024 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); mockNamespaces = mock(Namespaces.class); when(admin.namespaces()).thenReturn(mockNamespaces); @@ -416,7 +447,10 @@ public void namespaces() throws Exception { namespaces.run(split("set-backlog-quota myprop/clust/ns1 -p producer_exception -l 10G")); verify(mockNamespaces).setBacklogQuota("myprop/clust/ns1", - new BacklogQuota(10L * 1024 * 1024 * 1024, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10L * 1024 * 1024 * 1024) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); mockNamespaces = mock(Namespaces.class); when(admin.namespaces()).thenReturn(mockNamespaces); @@ -424,7 +458,11 @@ public void namespaces() throws Exception { namespaces.run(split("set-backlog-quota myprop/clust/ns1 -p producer_exception -l 10G -lt 10000")); verify(mockNamespaces).setBacklogQuota("myprop/clust/ns1", - new BacklogQuota(10l * 1024 * 1024 * 1024, 10000, RetentionPolicy.producer_exception)); + BacklogQuota.builder() + .limitSize(10l * 1024 * 1024 * 1024) + .limitTime(10000) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); namespaces.run(split("set-persistence myprop/clust/ns1 -e 2 -w 1 -a 1 -r 100.0")); verify(mockNamespaces).setPersistence("myprop/clust/ns1", @@ -458,14 +496,17 @@ public void namespaces() throws Exception { namespaces.run(split("set-auto-topic-creation myprop/clust/ns1 -e -t non-partitioned")); verify(mockNamespaces).setAutoTopicCreation("myprop/clust/ns1", - new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null)); + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.NON_PARTITIONED.toString()) + .build()); namespaces.run(split("remove-auto-topic-creation myprop/clust/ns1")); verify(mockNamespaces).removeAutoTopicCreation("myprop/clust/ns1"); namespaces.run(split("set-auto-subscription-creation myprop/clust/ns1 -e")); verify(mockNamespaces).setAutoSubscriptionCreation("myprop/clust/ns1", - new AutoSubscriptionCreationOverride(true)); + AutoSubscriptionCreationOverride.builder().allowAutoSubscriptionCreation(true).build()); namespaces.run(split("remove-auto-subscription-creation myprop/clust/ns1")); verify(mockNamespaces).removeAutoSubscriptionCreation("myprop/clust/ns1"); @@ -501,7 +542,7 @@ public void namespaces() throws Exception { namespaces.run(split("set-delayed-delivery myprop/clust/ns1 -e -t 1s")); verify(mockNamespaces).setDelayedDeliveryMessages("myprop/clust/ns1", - new DelayedDeliveryPolicies(1000, true)); + DelayedDeliveryPolicies.builder().tickTime(1000).active(true).build()); namespaces.run(split("get-delayed-delivery myprop/clust/ns1")); verify(mockNamespaces).getDelayedDelivery("myprop/clust/ns1"); @@ -606,7 +647,11 @@ public void namespaces() throws Exception { namespaces = new CmdNamespaces(() -> admin); namespaces.run(split("set-dispatch-rate myprop/clust/ns1 -md -1 -bd -1 -dt 2")); - verify(mockNamespaces).setDispatchRate("myprop/clust/ns1", new DispatchRate(-1, -1, 2)); + verify(mockNamespaces).setDispatchRate("myprop/clust/ns1", DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(2) + .build()); namespaces.run(split("get-dispatch-rate myprop/clust/ns1")); verify(mockNamespaces).getDispatchRate("myprop/clust/ns1"); @@ -633,7 +678,11 @@ public void namespaces() throws Exception { verify(mockNamespaces).removeSubscribeRate("myprop/clust/ns1"); namespaces.run(split("set-subscription-dispatch-rate myprop/clust/ns1 -md -1 -bd -1 -dt 2")); - verify(mockNamespaces).setSubscriptionDispatchRate("myprop/clust/ns1", new DispatchRate(-1, -1, 2)); + verify(mockNamespaces).setSubscriptionDispatchRate("myprop/clust/ns1", DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(2) + .build()); namespaces.run(split("get-subscription-dispatch-rate myprop/clust/ns1")); verify(mockNamespaces).getSubscriptionDispatchRate("myprop/clust/ns1"); @@ -738,7 +787,7 @@ public void namespacesCreateWithBundlesAndClusters() throws Exception { namespaces.run(split("create my-prop/my-namespace --bundles 5 --clusters a,b,c")); Policies policies = new Policies(); - policies.bundles = new BundlesData(5); + policies.bundles = BundlesData.builder().numBundles(5).build(); policies.replication_clusters = Sets.newHashSet("a", "b", "c"); verify(mockNamespaces).createNamespace("my-prop/my-namespace", policies); } @@ -854,8 +903,12 @@ public void topics() throws Exception { cmdTopics.run(split("get-backlog-quotas persistent://myprop/clust/ns1/ds1 -ap")); verify(mockTopics).getBacklogQuotaMap("persistent://myprop/clust/ns1/ds1", true); cmdTopics.run(split("set-backlog-quota persistent://myprop/clust/ns1/ds1 -l 10 -lt 1000 -p producer_request_hold")); - verify(mockTopics).setBacklogQuota("persistent://myprop/clust/ns1/ds1" - , new BacklogQuota(10L, 1000, BacklogQuota.RetentionPolicy.producer_request_hold)); + verify(mockTopics).setBacklogQuota("persistent://myprop/clust/ns1/ds1", + BacklogQuota.builder() + .limitSize(10) + .limitTime(1000) + .retentionPolicy(RetentionPolicy.producer_request_hold) + .build()); cmdTopics.run(split("remove-backlog-quota persistent://myprop/clust/ns1/ds1")); verify(mockTopics).removeBacklogQuota("persistent://myprop/clust/ns1/ds1"); @@ -939,7 +992,12 @@ public void topics() throws Exception { verify(mockTopics).setDeduplicationStatus("persistent://myprop/clust/ns1/ds1", false); cmdTopics.run(split("set-subscription-dispatch-rate persistent://myprop/clust/ns1/ds1 -md -1 -bd -1 -dt 2")); - verify(mockTopics).setSubscriptionDispatchRate("persistent://myprop/clust/ns1/ds1", new DispatchRate(-1, -1, 2)); + verify(mockTopics).setSubscriptionDispatchRate("persistent://myprop/clust/ns1/ds1", + DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(2) + .build()); cmdTopics.run(split("get-subscription-dispatch-rate persistent://myprop/clust/ns1/ds1")); verify(mockTopics).getSubscriptionDispatchRate("persistent://myprop/clust/ns1/ds1", false); cmdTopics.run(split("remove-subscription-dispatch-rate persistent://myprop/clust/ns1/ds1")); @@ -960,7 +1018,11 @@ public void topics() throws Exception { cmdTopics.run(split("set-replicator-dispatch-rate persistent://myprop/clust/ns1/ds1 -md 10 -bd 11 -dt 12")); verify(mockTopics).setReplicatorDispatchRate("persistent://myprop/clust/ns1/ds1", - new DispatchRate(10,11,12)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(10) + .dispatchThrottlingRateInByte(11) + .ratePeriodInSecond(12) + .build()); cmdTopics.run(split("remove-replicator-dispatch-rate persistent://myprop/clust/ns1/ds1")); verify(mockTopics).removeReplicatorDispatchRate("persistent://myprop/clust/ns1/ds1"); @@ -980,7 +1042,7 @@ public void topics() throws Exception { verify(mockTopics).getDelayedDeliveryPolicy("persistent://myprop/clust/ns1/ds1", false); cmdTopics.run(split("set-delayed-delivery persistent://myprop/clust/ns1/ds1 -t 10s --enable")); verify(mockTopics).setDelayedDeliveryPolicy("persistent://myprop/clust/ns1/ds1", - new DelayedDeliveryPolicies(10000, true)); + DelayedDeliveryPolicies.builder().tickTime(10000).active(true).build()); cmdTopics.run(split("remove-delayed-delivery persistent://myprop/clust/ns1/ds1")); verify(mockTopics).removeDelayedDeliveryPolicy("persistent://myprop/clust/ns1/ds1") ; @@ -1064,7 +1126,11 @@ public void topics() throws Exception { cmdTopics.run(split("remove-dispatch-rate persistent://myprop/clust/ns1/ds1")); verify(mockTopics).removeRetention("persistent://myprop/clust/ns1/ds1"); cmdTopics.run(split("set-dispatch-rate persistent://myprop/clust/ns1/ds1 -md -1 -bd -1 -dt 2")); - verify(mockTopics).setDispatchRate("persistent://myprop/clust/ns1/ds1", new DispatchRate(-1, -1, 2)); + verify(mockTopics).setDispatchRate("persistent://myprop/clust/ns1/ds1", DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(2) + .build()); cmdTopics.run(split("get-max-producers persistent://myprop/clust/ns1/ds1")); verify(mockTopics).getMaxProducers("persistent://myprop/clust/ns1/ds1", false); @@ -1324,7 +1390,7 @@ public void bookies() throws Exception { PulsarAdmin admin = Mockito.mock(PulsarAdmin.class); Bookies mockBookies = mock(Bookies.class); doReturn(mockBookies).when(admin).bookies(); - doReturn(new BookiesClusterInfo(Collections.emptyList())).when(mockBookies).getBookies(); + doReturn(BookiesClusterInfo.builder().bookies(Collections.emptyList()).build()).when(mockBookies).getBookies(); doReturn(new BookiesRackConfiguration()).when(mockBookies).getBookiesRackInfo(); CmdBookies bookies = new CmdBookies(() -> admin); @@ -1342,7 +1408,11 @@ public void bookies() throws Exception { verify(mockBookies).deleteBookieRackInfo("my-bookie:3181"); bookies.run(split("set-bookie-rack --group my-group --bookie my-bookie:3181 --rack rack-1 --hostname host-1")); - verify(mockBookies).updateBookieRackInfo("my-bookie:3181", "my-group", new BookieInfo("rack-1", "host-1")); + verify(mockBookies).updateBookieRackInfo("my-bookie:3181", "my-group", + BookieInfo.builder() + .rack("rack-1") + .hostname("host-1") + .build()); } @Test diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index a094ba7479c4b..e6a4ea67199f0 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -38,6 +38,11 @@ jcommander compile + + ${project.groupId} + pulsar-client-admin-api + ${project.version} + ${project.groupId} pulsar-client-admin-original diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBookies.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBookies.java index 133f3335e7985..e6551b9eb5230 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBookies.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBookies.java @@ -87,7 +87,11 @@ private class UpdateBookie extends CliCommand { @Override void run() throws Exception { - getAdmin().bookies().updateBookieRackInfo(bookieAddress, group, new BookieInfo(bookieRack, bookieHost)); + getAdmin().bookies().updateBookieRackInfo(bookieAddress, group, + BookieInfo.builder() + .rack(bookieRack) + .hostname(bookieHost) + .build()); } } 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 060f9e69306a2..e3b3c5e24a1a1 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 @@ -19,23 +19,22 @@ package org.apache.pulsar.admin.cli; import static org.apache.commons.lang3.StringUtils.isNotBlank; - import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; import com.google.common.collect.Sets; import java.util.Arrays; import java.util.function.Supplier; +import lombok.Getter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.admin.cli.utils.CmdUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.ProxyProtocol; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FailureDomainImpl; -import org.apache.pulsar.common.policies.data.ClusterData; -import lombok.Getter; - @Parameters(commandDescription = "Operations about clusters") public class CmdClusters extends CmdBase { @@ -146,17 +145,17 @@ void run() throws PulsarAdminException { @Parameters(commandDescription = "Get list of peer-clusters") private class GetPeerClusters extends CliCommand { - + @Parameter(description = "cluster-name", required = true) private java.util.List params; - + void run() throws PulsarAdminException { String cluster = getOneArgument(params); print(getAdmin().clusters().getPeerClusterNames(cluster)); } } - - + + @Parameters(commandDescription = "Create a new failure-domain for a cluster. updates it if already created.") private class CreateFailureDomain extends CliCommand { @Parameter(description = "cluster-name", required = true) @@ -167,11 +166,12 @@ private class CreateFailureDomain extends CliCommand { @Parameter(names = "--broker-list", description = "Comma separated broker list", required = false) private String brokerList; - + void run() throws PulsarAdminException { String cluster = getOneArgument(params); - FailureDomainImpl domain = new FailureDomainImpl(); - domain.setBrokers((isNotBlank(brokerList) ? Sets.newHashSet(brokerList.split(",")): null)); + FailureDomain domain = FailureDomainImpl.builder() + .brokers((isNotBlank(brokerList) ? Sets.newHashSet(brokerList.split(",")) : null)) + .build(); getAdmin().clusters().createFailureDomain(cluster, domainName, domain); } } @@ -189,12 +189,13 @@ private class UpdateFailureDomain extends CliCommand { void run() throws PulsarAdminException { String cluster = getOneArgument(params); - FailureDomainImpl domain = new FailureDomainImpl(); - domain.setBrokers((isNotBlank(brokerList) ? Sets.newHashSet(brokerList.split(",")) : null)); + FailureDomain domain = FailureDomainImpl.builder() + .brokers((isNotBlank(brokerList) ? Sets.newHashSet(brokerList.split(",")) : null)) + .build(); getAdmin().clusters().updateFailureDomain(cluster, domainName, domain); } } - + @Parameters(commandDescription = "Deletes an existing failure-domain") private class DeleteFailureDomain extends CliCommand { @Parameter(description = "cluster-name", required = true) @@ -211,10 +212,10 @@ void run() throws PulsarAdminException { @Parameters(commandDescription = "List the existing failure-domains for a cluster") private class ListFailureDomains extends CliCommand { - + @Parameter(description = "cluster-name", required = true) private java.util.List params; - + void run() throws PulsarAdminException { String cluster = getOneArgument(params); print(getAdmin().clusters().getFailureDomains(cluster)); @@ -225,7 +226,7 @@ void run() throws PulsarAdminException { private class GetFailureDomain extends CliCommand { @Parameter(description = "cluster-name", required = true) private java.util.List params; - + @Parameter(names = "--domain-name", description = "domain-name", required = true) private String domainName; @@ -322,64 +323,68 @@ abstract class ClusterDetailsCommand extends BaseCommand { void processArguments() throws Exception { super.processArguments(); + ClusterData.Builder builder; if (null != clusterConfigFile) { - this.clusterData = CmdUtils.loadConfig(clusterConfigFile, ClusterDataImpl.class); + builder = CmdUtils.loadConfig(clusterConfigFile, ClusterDataImpl.ClusterDataImplBuilder.class); } else { - this.clusterData = new ClusterDataImpl(); + builder = ClusterData.builder(); } if (serviceUrl != null) { - clusterData.setServiceUrl(serviceUrl); + builder.serviceUrl(serviceUrl); } if (serviceUrlTls != null) { - clusterData.setServiceUrlTls(serviceUrlTls); + builder.serviceUrlTls(serviceUrlTls); } if (brokerServiceUrl != null) { - clusterData.setBrokerServiceUrl(brokerServiceUrl); + builder.brokerServiceUrl(brokerServiceUrl); } if (brokerServiceUrlTls != null) { - clusterData.setBrokerServiceUrlTls(brokerServiceUrlTls); + builder.brokerServiceUrlTls(brokerServiceUrlTls); } if (proxyServiceUrl != null) { - clusterData.setProxyServiceUrl(proxyServiceUrl); + builder.proxyServiceUrl(proxyServiceUrl); } if (authenticationPlugin != null) { - clusterData.setAuthenticationPlugin(authenticationPlugin); + builder.authenticationPlugin(authenticationPlugin); } if (authenticationParameters != null) { - clusterData.setAuthenticationParameters(authenticationParameters); + builder.authenticationParameters(authenticationParameters); } if (proxyProtocol != null) { - clusterData.setProxyProtocol(proxyProtocol); + builder.proxyProtocol(proxyProtocol); } if (brokerClientTlsEnabled != null) { - clusterData.setBrokerClientTlsEnabled(brokerClientTlsEnabled); + builder.brokerClientTlsEnabled(brokerClientTlsEnabled); } if (tlsAllowInsecureConnection != null) { - clusterData.setTlsAllowInsecureConnection(tlsAllowInsecureConnection); + builder.tlsAllowInsecureConnection(tlsAllowInsecureConnection); } if (brokerClientTlsEnabledWithKeyStore != null) { - clusterData.setBrokerClientTlsEnabledWithKeyStore(brokerClientTlsEnabledWithKeyStore); + builder.brokerClientTlsEnabledWithKeyStore(brokerClientTlsEnabledWithKeyStore); } if (brokerClientTlsTrustStoreType != null) { - clusterData.setBrokerClientTlsTrustStoreType(brokerClientTlsTrustStoreType); + builder.brokerClientTlsTrustStoreType(brokerClientTlsTrustStoreType); } if (brokerClientTlsTrustStore != null) { - clusterData.setBrokerClientTlsTrustStore(brokerClientTlsTrustStore); + builder.brokerClientTlsTrustStore(brokerClientTlsTrustStore); } if (brokerClientTlsTrustStorePassword != null) { - clusterData.setBrokerClientTlsTrustStorePassword(brokerClientTlsTrustStorePassword); + builder.brokerClientTlsTrustStorePassword(brokerClientTlsTrustStorePassword); } if (brokerClientTrustCertsFilePath != null) { - clusterData.setBrokerClientTrustCertsFilePath(brokerClientTrustCertsFilePath); + builder.brokerClientTrustCertsFilePath(brokerClientTrustCertsFilePath); } + if (listenerName != null) { - clusterData.setListenerName(listenerName); + builder.listenerName(listenerName); } + + this.clusterData = builder.build(); validateClusterData(clusterData); } } - + public CmdClusters(Supplier admin) { super("clusters", admin); jcommander.addCommand("get", new Get()); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java index 73b066c429ce6..1f8df532b1feb 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java @@ -26,6 +26,7 @@ import org.apache.pulsar.admin.cli.utils.NameValueParameterSplitter; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; @@ -65,7 +66,7 @@ void run() throws PulsarAdminException { String policyName = getOneArgument(params, 1, 2); // validate and create the POJO - NamespaceIsolationDataImpl namespaceIsolationData = createNamespaceIsolationData(namespaces, primary, secondary, + NamespaceIsolationData namespaceIsolationData = createNamespaceIsolationData(namespaces, primary, secondary, autoFailoverPolicyTypeName, autoFailoverPolicyParams); getAdmin().clusters().createNamespaceIsolationPolicy(clusterName, policyName, namespaceIsolationData); @@ -159,7 +160,7 @@ private List validateList(List list) { return list; } - private NamespaceIsolationDataImpl createNamespaceIsolationData(List namespaces, List primary, + private NamespaceIsolationData createNamespaceIsolationData(List namespaces, List primary, List secondary, String autoFailoverPolicyTypeName, Map autoFailoverPolicyParams) { // validate @@ -181,27 +182,29 @@ private NamespaceIsolationDataImpl createNamespaceIsolationData(List nam // System.out.println("autoFailoverPolicyTypeName = " + autoFailoverPolicyTypeName); // System.out.println("autoFailoverPolicyParams = " + autoFailoverPolicyParams); - NamespaceIsolationDataImpl nsIsolationData = new NamespaceIsolationDataImpl(); + NamespaceIsolationData.Builder nsIsolationDataBuilder = NamespaceIsolationData.builder(); if (namespaces != null) { - nsIsolationData.namespaces = namespaces; + nsIsolationDataBuilder.namespaces(namespaces); } if (primary != null) { - nsIsolationData.primary = primary; + nsIsolationDataBuilder.primary(primary); } if (secondary != null) { - nsIsolationData.secondary = secondary; + nsIsolationDataBuilder.secondary(secondary); } - nsIsolationData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsIsolationData.auto_failover_policy.policy_type = AutoFailoverPolicyType - .fromString(autoFailoverPolicyTypeName); - nsIsolationData.auto_failover_policy.parameters = autoFailoverPolicyParams; + AutoFailoverPolicyType policyType = AutoFailoverPolicyType.fromString(autoFailoverPolicyTypeName); + + nsIsolationDataBuilder.autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(policyType) + .parameters(autoFailoverPolicyParams) + .build()); // validation if necessary - if (nsIsolationData.auto_failover_policy.policy_type == AutoFailoverPolicyType.min_available) { + if (policyType == AutoFailoverPolicyType.min_available) { // ignore boolean error = true; String[] expectParamKeys = { "min_limit", "usage_threshold" }; @@ -225,7 +228,7 @@ private NamespaceIsolationDataImpl createNamespaceIsolationData(List nam throw new ParameterException("Unknown auto failover policy type specified : " + autoFailoverPolicyTypeName); } - return nsIsolationData; + return nsIsolationDataBuilder.build(); } public CmdNamespaceIsolationPolicy(Supplier admin) { diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index 6049f77de76d2..bae1ae211f066 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -52,6 +52,7 @@ import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; +import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.OffloadedReadPriority; import org.apache.pulsar.common.policies.data.PersistencePolicies; @@ -163,7 +164,8 @@ void run() throws PulsarAdminException { NamespaceName namespaceName = NamespaceName.get(namespace); if (namespaceName.isV2()) { Policies policies = new Policies(); - policies.bundles = numBundles > 0 ? new BundlesData(numBundles) : null; + policies.bundles = numBundles > 0 ? BundlesData.builder() + .numBundles(numBundles).build() : null; if (clusters != null) { policies.replication_clusters = new HashSet<>(clusters); @@ -563,7 +565,12 @@ void run() throws PulsarAdminException { throw new ParameterException("Must specify num-partitions > 0 for partitioned topic type."); } } - getAdmin().namespaces().setAutoTopicCreation(namespace, new AutoTopicCreationOverride(enable, type, defaultNumPartitions)); + getAdmin().namespaces().setAutoTopicCreation(namespace, + AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(enable) + .topicType(type) + .defaultNumPartitions(defaultNumPartitions) + .build()); } } @@ -591,7 +598,10 @@ private class SetAutoSubscriptionCreation extends CliCommand { @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - getAdmin().namespaces().setAutoSubscriptionCreation(namespace, new AutoSubscriptionCreationOverride(enable)); + getAdmin().namespaces().setAutoSubscriptionCreation(namespace, + AutoSubscriptionCreationOverride.builder() + .allowAutoSubscriptionCreation(enable) + .build()); } } @@ -685,7 +695,10 @@ private class SetBookieAffinityGroup extends CliCommand { void run() throws PulsarAdminException { String namespace = validateNamespace(params); getAdmin().namespaces().setBookieAffinityGroup(namespace, - new BookieAffinityGroupData(bookieAffinityGroupNamePrimary, bookieAffinityGroupNameSecondary)); + BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary(bookieAffinityGroupNamePrimary) + .bookkeeperAffinityGroupSecondary(bookieAffinityGroupNameSecondary) + .build()); } } @@ -805,7 +818,12 @@ private class SetDispatchRate extends CliCommand { void run() throws PulsarAdminException { String namespace = validateNamespace(params); getAdmin().namespaces().setDispatchRate(namespace, - new DispatchRate(msgDispatchRate, byteDispatchRate, dispatchRatePeriodSec, relativeToPublishRate)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(msgDispatchRate) + .dispatchThrottlingRateInByte(byteDispatchRate) + .ratePeriodInSecond(dispatchRatePeriodSec) + .relativeToPublishRate(relativeToPublishRate) + .build()); } } @@ -905,7 +923,12 @@ private class SetSubscriptionDispatchRate extends CliCommand { void run() throws PulsarAdminException { String namespace = validateNamespace(params); getAdmin().namespaces().setSubscriptionDispatchRate(namespace, - new DispatchRate(msgDispatchRate, byteDispatchRate, dispatchRatePeriodSec, relativeToPublishRate)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(msgDispatchRate) + .dispatchThrottlingRateInByte(byteDispatchRate) + .ratePeriodInSecond(dispatchRatePeriodSec) + .relativeToPublishRate(relativeToPublishRate) + .build()); } } @@ -1000,7 +1023,11 @@ private class SetReplicatorDispatchRate extends CliCommand { void run() throws PulsarAdminException { String namespace = validateNamespace(params); getAdmin().namespaces().setReplicatorDispatchRate(namespace, - new DispatchRate(msgDispatchRate, byteDispatchRate, dispatchRatePeriodSec)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(msgDispatchRate) + .dispatchThrottlingRateInByte(byteDispatchRate) + .ratePeriodInSecond(dispatchRatePeriodSec) + .build()); } } @@ -1071,7 +1098,11 @@ void run() throws PulsarAdminException { limit = validateSizeString(limitStr); String namespace = validateNamespace(params); - getAdmin().namespaces().setBacklogQuota(namespace, new BacklogQuota(limit, limitTime, policy)); + getAdmin().namespaces().setBacklogQuota(namespace, + BacklogQuota.builder().limitSize(limit) + .limitTime(limitTime) + .retentionPolicy(policy) + .build()); } } @@ -1330,7 +1361,10 @@ void run() throws PulsarAdminException { throw new ParameterException("Need to specify either --enable or --disable"); } - getAdmin().namespaces().setDelayedDeliveryMessages(namespace, new DelayedDeliveryPolicies(delayedDeliveryTimeInMills, enable)); + getAdmin().namespaces().setDelayedDeliveryMessages(namespace, DelayedDeliveryPolicies.builder() + .tickTime(delayedDeliveryTimeInMills) + .active(enable) + .build()); } } @@ -2030,7 +2064,7 @@ && maxValueCheck("OffloadAfterThreshold", offloadAfterThreshold, Long.MAX_VALUE) } } - OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create(driver, region, bucket, endpoint, + OffloadPolicies offloadPolicies = OffloadPoliciesImpl.create(driver, region, bucket, endpoint, s3Role, s3RoleSessionName, awsId, awsSecret, maxBlockSizeInBytes, readBufferSizeInBytes, offloadAfterThresholdInBytes, 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 0e0e413722d7f..64fe0178a5f5b 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 @@ -1119,7 +1119,11 @@ void run() throws PulsarAdminException { limit = validateSizeString(limitStr); String persistentTopic = validatePersistentTopic(params); - getTopics().setBacklogQuota(persistentTopic, new BacklogQuota(limit, limitTime, policy)); + getTopics().setBacklogQuota(persistentTopic, BacklogQuota.builder() + .limitSize(limit) + .limitTime(limitTime) + .retentionPolicy(policy) + .build()); } } @@ -1175,7 +1179,10 @@ void run() throws PulsarAdminException { throw new ParameterException("Need to specify either --enable or --disable"); } - getTopics().setDelayedDeliveryPolicy(topicName, new DelayedDeliveryPolicies(delayedDeliveryTimeInMills, enable)); + getTopics().setDelayedDeliveryPolicy(topicName, DelayedDeliveryPolicies.builder() + .tickTime(delayedDeliveryTimeInMills) + .active(enable) + .build()); } } @@ -1628,7 +1635,12 @@ private class SetDispatchRate extends CliCommand { void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); getTopics().setDispatchRate(persistentTopic, - new DispatchRate(msgDispatchRate, byteDispatchRate, dispatchRatePeriodSec, relativeToPublishRate)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(msgDispatchRate) + .dispatchThrottlingRateInByte(byteDispatchRate) + .ratePeriodInSecond(dispatchRatePeriodSec) + .relativeToPublishRate(relativeToPublishRate) + .build()); } } @@ -1889,7 +1901,12 @@ private class SetSubscriptionDispatchRate extends CliCommand { void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); getTopics().setSubscriptionDispatchRate(persistentTopic, - new DispatchRate(msgDispatchRate, byteDispatchRate, dispatchRatePeriodSec, relativeToPublishRate)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(msgDispatchRate) + .dispatchThrottlingRateInByte(byteDispatchRate) + .ratePeriodInSecond(dispatchRatePeriodSec) + .relativeToPublishRate(relativeToPublishRate) + .build()); } } @@ -1945,7 +1962,12 @@ private class SetReplicatorDispatchRate extends CliCommand { void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); getTopics().setReplicatorDispatchRate(persistentTopic, - new DispatchRate(msgDispatchRate, byteDispatchRate, dispatchRatePeriodSec, relativeToPublishRate)); + DispatchRate.builder() + .dispatchThrottlingRateInMsg(msgDispatchRate) + .dispatchThrottlingRateInByte(byteDispatchRate) + .ratePeriodInSecond(dispatchRatePeriodSec) + .relativeToPublishRate(relativeToPublishRate) + .build()); } } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java index c86ea4230151c..ad5e39e2ce343 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java @@ -26,12 +26,12 @@ import java.io.File; import java.io.IOException; +import org.apache.pulsar.common.util.ObjectMapperFactory; public class CmdUtils { public static T loadConfig(String file, Class clazz) throws IOException { - ObjectMapper mapper = new ObjectMapper(new YAMLFactory()); try { - return mapper.readValue(new File(file), clazz); + return ObjectMapperFactory.getThreadLocalYaml().readValue(new File(file), clazz); } catch (Exception ex) { if (ex instanceof UnrecognizedPropertyException) { UnrecognizedPropertyException unrecognizedPropertyException 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 d7f6fcb16a550..0ab093d56c26b 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 @@ -34,6 +34,7 @@ import org.apache.pulsar.client.admin.Clusters; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -63,8 +64,8 @@ public void testCmdClusterConfigFile() throws Exception { public void testCmdClusterConfigFile(ClusterData testClusterData, ClusterData expectedClusterData) throws Exception { File file = Files.createTempFile("tmp_cluster", ".yaml").toFile(); - new YAMLMapper().writeValue(file, testClusterData); - Assert.assertEquals(testClusterData, CmdUtils.loadConfig(file.getAbsolutePath(), ClusterDataImpl.class)); + ObjectMapperFactory.getThreadLocalYaml().writeValue(file, testClusterData); + Assert.assertEquals(testClusterData, CmdUtils.loadConfig(file.getAbsolutePath(), ClusterData.class)); // test create cluster cmdClusters.run(new String[]{"create", "test_cluster", "--cluster-config-file", file.getAbsolutePath()}); @@ -75,7 +76,7 @@ public void testCmdClusterConfigFile(ClusterData testClusterData, ClusterData ex } public ClusterData buildClusterData() { - return ClusterDataImpl.builder() + return ClusterData.builder() .serviceUrlTls("https://my-service.url:4443") .authenticationPlugin("authenticationPlugin") .authenticationParameters("authenticationParameters") diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 414397d198248..9b0b9299bf403 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -156,6 +156,13 @@ commons-io + + com.github.spotbugs + spotbugs-annotations + provided + true + + org.lz4 lz4-java diff --git a/pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/AuthPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/AuthPoliciesImpl.java new file mode 100644 index 0000000000000..0f0429ea360de --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/AuthPoliciesImpl.java @@ -0,0 +1,78 @@ +/** + * 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.admin.internal.data; + +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.Value; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AuthPolicies; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public final class AuthPoliciesImpl implements AuthPolicies { + + @JsonProperty("namespace_auth") + private Map> namespaceAuthentication = new TreeMap<>(); + + @JsonProperty("destination_auth") + private Map>> topicAuthentication = new TreeMap<>(); + + @JsonProperty("subscription_auth_roles") + private Map> subscriptionAuthentication = new TreeMap<>(); + + public static AuthPolicies.Builder builder() { + return new AuthPoliciesImplBuilder(); + } + + private static class AuthPoliciesImplBuilder implements AuthPolicies.Builder { + private Map> namespaceAuthentication = new TreeMap<>(); + private Map>> topicAuthentication = new TreeMap<>(); + private Map> subscriptionAuthentication = new TreeMap<>(); + + AuthPoliciesImplBuilder() { + } + + public AuthPoliciesImplBuilder namespaceAuthentication(Map> namespaceAuthentication) { + this.namespaceAuthentication = namespaceAuthentication; + return this; + } + + public AuthPoliciesImplBuilder topicAuthentication( + Map>> topicAuthentication) { + this.topicAuthentication = topicAuthentication; + return this; + } + + public AuthPoliciesImplBuilder subscriptionAuthentication(Map> subscriptionAuthentication) { + this.subscriptionAuthentication = subscriptionAuthentication; + return this; + } + + public AuthPoliciesImpl build() { + return new AuthPoliciesImpl(namespaceAuthentication, topicAuthentication, subscriptionAuthentication); + } + } +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/package-info.java b/pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/package-info.java new file mode 100644 index 0000000000000..b93a6309ae8fc --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/client/admin/internal/data/package-info.java @@ -0,0 +1,19 @@ +/** + * 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.admin.internal.data; \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataImpl.java index ce17dac25e7c4..e0cbde9614518 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataImpl.java @@ -19,11 +19,13 @@ package org.apache.pulsar.common.policies.data; import static com.google.common.base.Preconditions.checkArgument; -import com.google.common.base.Objects; +import com.fasterxml.jackson.annotation.JsonProperty; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; import java.util.Map; +import lombok.AllArgsConstructor; import lombok.Data; +import lombok.NoArgsConstructor; import org.apache.pulsar.common.policies.impl.AutoFailoverPolicyFactory; /** @@ -34,14 +36,17 @@ description = "The auto failover policy configuration data" ) @Data +@NoArgsConstructor +@AllArgsConstructor public class AutoFailoverPolicyDataImpl implements AutoFailoverPolicyData { @ApiModelProperty( name = "policy_type", value = "The auto failover policy type", allowableValues = "min_available" ) - @SuppressWarnings("checkstyle:MemberName") - public AutoFailoverPolicyType policy_type; + @JsonProperty("policy_type") + private AutoFailoverPolicyType policyType; + @ApiModelProperty( name = "parameters", value = @@ -57,30 +62,33 @@ public class AutoFailoverPolicyDataImpl implements AutoFailoverPolicyData { + " \"usage_threshold\": 80\n" + "}\n" ) - public Map parameters; - - @Override - public int hashCode() { - return Objects.hashCode(policy_type, parameters); - } + private Map parameters; - @Override - public boolean equals(Object obj) { - if (obj instanceof AutoFailoverPolicyDataImpl) { - AutoFailoverPolicyDataImpl other = (AutoFailoverPolicyDataImpl) obj; - return Objects.equal(policy_type, other.policy_type) && Objects.equal(parameters, other.parameters); - } - - return false; + public static AutoFailoverPolicyDataImplBuilder builder() { + return new AutoFailoverPolicyDataImplBuilder(); } public void validate() { - checkArgument(policy_type != null && parameters != null); + checkArgument(policyType != null && parameters != null); AutoFailoverPolicyFactory.create(this); } - @Override - public String toString() { - return String.format("policy_type=%s parameters=%s", policy_type, parameters); + public static class AutoFailoverPolicyDataImplBuilder implements AutoFailoverPolicyData.Builder { + private AutoFailoverPolicyType policyType; + private Map parameters; + + public AutoFailoverPolicyDataImplBuilder policyType(AutoFailoverPolicyType policyType) { + this.policyType = policyType; + return this; + } + + public AutoFailoverPolicyDataImplBuilder parameters(Map parameters) { + this.parameters = parameters; + return this; + } + + public AutoFailoverPolicyDataImpl build() { + return new AutoFailoverPolicyDataImpl(policyType, parameters); + } } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationDataImpl.java index 4cf23b4c4da9c..b0fc2277b5a78 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerNamespaceIsolationDataImpl.java @@ -18,11 +18,12 @@ */ package org.apache.pulsar.common.policies.data; -import com.google.common.base.Objects; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; -import lombok.Data; import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; /** * The namespace isolation data for a given broker. @@ -32,44 +33,67 @@ description = "The namespace isolation data for a given broker" ) @Data -public class BrokerNamespaceIsolationDataImpl implements BrokerNamespaceIsolationData { +@AllArgsConstructor +@NoArgsConstructor +public final class BrokerNamespaceIsolationDataImpl implements BrokerNamespaceIsolationData { @ApiModelProperty( name = "brokerName", value = "The broker name", example = "broker1:8080" ) - public String brokerName; + private String brokerName; @ApiModelProperty( name = "policyName", value = "Policy name", example = "my-policy" ) - public String policyName; + private String policyName; @ApiModelProperty( name = "isPrimary", value = "Is Primary broker", example = "true/false" ) - public boolean isPrimary; + private boolean isPrimary; @ApiModelProperty( name = "namespaceRegex", value = "The namespace-isolation policies attached to this broker" ) - public List namespaceRegex; //isolated namespace regex + private List namespaceRegex; //isolated namespace regex - @Override - public int hashCode() { - return Objects.hashCode(brokerName, namespaceRegex); + public static BrokerNamespaceIsolationDataImplBuilder builder() { + return new BrokerNamespaceIsolationDataImplBuilder(); } - @Override - public boolean equals(Object obj) { - if (obj instanceof BrokerNamespaceIsolationDataImpl) { - BrokerNamespaceIsolationDataImpl other = (BrokerNamespaceIsolationDataImpl) obj; - return Objects.equal(brokerName, other.brokerName) && Objects.equal(namespaceRegex, other.namespaceRegex); + + public static class BrokerNamespaceIsolationDataImplBuilder implements BrokerNamespaceIsolationData.Builder { + private String brokerName; + private String policyName; + private boolean isPrimary; + private List namespaceRegex; + + public BrokerNamespaceIsolationDataImplBuilder brokerName(String brokerName) { + this.brokerName = brokerName; + return this; + } + + public BrokerNamespaceIsolationDataImplBuilder policyName(String policyName) { + this.policyName = policyName; + return this; } - return false; - } + public BrokerNamespaceIsolationDataImplBuilder primary(boolean isPrimary) { + this.isPrimary = isPrimary; + return this; + } + + public BrokerNamespaceIsolationDataImplBuilder namespaceRegex(List namespaceRegex) { + this.namespaceRegex = namespaceRegex; + return this; + } + + public BrokerNamespaceIsolationDataImpl build() { + return new BrokerNamespaceIsolationDataImpl(brokerName, policyName, isPrimary, namespaceRegex); + } + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java index a974ee2a31b48..0c26e97d8d3e9 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java @@ -18,17 +18,12 @@ */ package org.apache.pulsar.common.policies.data; -import static com.google.common.base.Preconditions.checkNotNull; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; import java.util.LinkedHashSet; import lombok.AllArgsConstructor; -import lombok.Builder; import lombok.Data; -import lombok.Getter; import lombok.NoArgsConstructor; -import lombok.Setter; -import lombok.experimental.Accessors; import org.apache.pulsar.client.api.ProxyProtocol; /** @@ -39,11 +34,9 @@ description = "The configuration data for a cluster" ) @Data -@Builder @AllArgsConstructor @NoArgsConstructor -@Accessors(chain = true) -public class ClusterDataImpl implements ClusterData { +public final class ClusterDataImpl implements ClusterData, Cloneable { @ApiModelProperty( name = "serviceUrl", value = "The HTTP rest service URL (for admin operations)", @@ -106,26 +99,26 @@ public class ClusterDataImpl implements ClusterData { value = "Enable TLS when talking with other brokers in the same cluster (admin operation)" + " or different clusters (replication)" ) - private boolean brokerClientTlsEnabled = false; + private boolean brokerClientTlsEnabled; @ApiModelProperty( name = "tlsAllowInsecureConnection", value = "Allow TLS connections to servers whose certificate cannot be" + " be verified to have been signed by a trusted certificate" + " authority." ) - private boolean tlsAllowInsecureConnection = false; + private boolean tlsAllowInsecureConnection; @ApiModelProperty( name = "brokerClientTlsEnabledWithKeyStore", value = "Whether internal client use KeyStore type to authenticate with other Pulsar brokers" ) - private boolean brokerClientTlsEnabledWithKeyStore = false; + private boolean brokerClientTlsEnabledWithKeyStore; @ApiModelProperty( name = "brokerClientTlsTrustStoreType", value = "TLS TrustStore type configuration for internal client: JKS, PKCS12" + " used by the internal client to authenticate with Pulsar brokers", example = "JKS" ) - private String brokerClientTlsTrustStoreType = "JKS"; + private String brokerClientTlsTrustStoreType; @ApiModelProperty( name = "brokerClientTlsTrustStore", value = "TLS TrustStore path for internal client" @@ -150,74 +143,142 @@ public class ClusterDataImpl implements ClusterData { ) private String listenerName; - public ClusterDataImpl(String serviceUrl) { - this(serviceUrl, ""); + public static ClusterDataImplBuilder builder() { + return new ClusterDataImplBuilder(); } - public ClusterDataImpl(String serviceUrl, String serviceUrlTls) { - this.serviceUrl = serviceUrl; - this.serviceUrlTls = serviceUrlTls; + @Override + public ClusterDataImplBuilder clone() { + return builder() + .serviceUrl(serviceUrl) + .serviceUrlTls(serviceUrlTls) + .brokerServiceUrl(brokerServiceUrl) + .brokerServiceUrlTls(brokerServiceUrlTls) + .proxyServiceUrl(proxyServiceUrl) + .authenticationPlugin(authenticationPlugin) + .authenticationParameters(authenticationParameters) + .proxyProtocol(proxyProtocol) + .peerClusterNames(peerClusterNames) + .brokerClientTlsEnabled(brokerClientTlsEnabled) + .tlsAllowInsecureConnection(tlsAllowInsecureConnection) + .brokerClientTlsEnabledWithKeyStore(brokerClientTlsEnabledWithKeyStore) + .brokerClientTlsTrustStoreType(brokerClientTlsTrustStoreType) + .brokerClientTlsTrustStore(brokerClientTlsTrustStore) + .brokerClientTlsTrustStorePassword(brokerClientTlsTrustStorePassword) + .brokerClientTrustCertsFilePath(brokerClientTrustCertsFilePath) + .listenerName(listenerName); } - public ClusterDataImpl(String serviceUrl, String serviceUrlTls, String brokerServiceUrl, String brokerServiceUrlTls) { - this.serviceUrl = serviceUrl; - this.serviceUrlTls = serviceUrlTls; - this.brokerServiceUrl = brokerServiceUrl; - this.brokerServiceUrlTls = brokerServiceUrlTls; - } + @Data + public static class ClusterDataImplBuilder implements ClusterData.Builder { + private String serviceUrl; + private String serviceUrlTls; + private String brokerServiceUrl; + private String brokerServiceUrlTls; + private String proxyServiceUrl; + private String authenticationPlugin; + private String authenticationParameters; + private ProxyProtocol proxyProtocol; + private LinkedHashSet peerClusterNames; + private boolean brokerClientTlsEnabled = false; + private boolean tlsAllowInsecureConnection = false; + private boolean brokerClientTlsEnabledWithKeyStore = false; + private String brokerClientTlsTrustStoreType = "JKS"; + private String brokerClientTlsTrustStore; + private String brokerClientTlsTrustStorePassword; + private String brokerClientTrustCertsFilePath; + private String listenerName; - public ClusterDataImpl(String serviceUrl, String serviceUrlTls, String brokerServiceUrl, String brokerServiceUrlTls, - String authenticationPlugin, String authenticationParameters) { - this.serviceUrl = serviceUrl; - this.serviceUrlTls = serviceUrlTls; - this.brokerServiceUrl = brokerServiceUrl; - this.brokerServiceUrlTls = brokerServiceUrlTls; - this.authenticationPlugin = authenticationPlugin; - this.authenticationParameters = authenticationParameters; - } + ClusterDataImplBuilder() { + } - public ClusterDataImpl(String serviceUrl, String serviceUrlTls, String brokerServiceUrl, String brokerServiceUrlTls, - String authenticationPlugin, String authenticationParameters, String listenerName) { - this.serviceUrl = serviceUrl; - this.serviceUrlTls = serviceUrlTls; - this.brokerServiceUrl = brokerServiceUrl; - this.brokerServiceUrlTls = brokerServiceUrlTls; - this.authenticationPlugin = authenticationPlugin; - this.authenticationParameters = authenticationParameters; - this.listenerName = listenerName; - } + public ClusterDataImplBuilder serviceUrl(String serviceUrl) { + this.serviceUrl = serviceUrl; + return this; + } - public ClusterDataImpl(String serviceUrl, String serviceUrlTls, String brokerServiceUrl, String brokerServiceUrlTls, - String proxyServiceUrl, String authenticationPlugin, String authenticationParameters, - ProxyProtocol proxyProtocol) { - this.serviceUrl = serviceUrl; - this.serviceUrlTls = serviceUrlTls; - this.brokerServiceUrl = brokerServiceUrl; - this.brokerServiceUrlTls = brokerServiceUrlTls; - this.authenticationPlugin = authenticationPlugin; - this.authenticationParameters = authenticationParameters; - this.proxyServiceUrl = proxyServiceUrl; - this.proxyProtocol = proxyProtocol; - } + public ClusterDataImplBuilder serviceUrlTls(String serviceUrlTls) { + this.serviceUrlTls = serviceUrlTls; + return this; + } + + public ClusterDataImplBuilder brokerServiceUrl(String brokerServiceUrl) { + this.brokerServiceUrl = brokerServiceUrl; + return this; + } + + public ClusterDataImplBuilder brokerServiceUrlTls(String brokerServiceUrlTls) { + this.brokerServiceUrlTls = brokerServiceUrlTls; + return this; + } + + public ClusterDataImplBuilder proxyServiceUrl(String proxyServiceUrl) { + this.proxyServiceUrl = proxyServiceUrl; + return this; + } + + public ClusterDataImplBuilder authenticationPlugin(String authenticationPlugin) { + this.authenticationPlugin = authenticationPlugin; + return this; + } + + public ClusterDataImplBuilder authenticationParameters(String authenticationParameters) { + this.authenticationParameters = authenticationParameters; + return this; + } + + public ClusterDataImplBuilder proxyProtocol(ProxyProtocol proxyProtocol) { + this.proxyProtocol = proxyProtocol; + return this; + } + + public ClusterDataImplBuilder peerClusterNames(LinkedHashSet peerClusterNames) { + this.peerClusterNames = peerClusterNames; + return this; + } + + public ClusterDataImplBuilder brokerClientTlsEnabled(boolean brokerClientTlsEnabled) { + this.brokerClientTlsEnabled = brokerClientTlsEnabled; + return this; + } + + public ClusterDataImplBuilder tlsAllowInsecureConnection(boolean tlsAllowInsecureConnection) { + this.tlsAllowInsecureConnection = tlsAllowInsecureConnection; + return this; + } + + public ClusterDataImplBuilder brokerClientTlsEnabledWithKeyStore(boolean brokerClientTlsEnabledWithKeyStore) { + this.brokerClientTlsEnabledWithKeyStore = brokerClientTlsEnabledWithKeyStore; + return this; + } + + public ClusterDataImplBuilder brokerClientTlsTrustStoreType(String brokerClientTlsTrustStoreType) { + this.brokerClientTlsTrustStoreType = brokerClientTlsTrustStoreType; + return this; + } + + public ClusterDataImplBuilder brokerClientTlsTrustStore(String brokerClientTlsTrustStore) { + this.brokerClientTlsTrustStore = brokerClientTlsTrustStore; + return this; + } + + public ClusterDataImplBuilder brokerClientTlsTrustStorePassword(String brokerClientTlsTrustStorePassword) { + this.brokerClientTlsTrustStorePassword = brokerClientTlsTrustStorePassword; + return this; + } + + public ClusterDataImplBuilder brokerClientTrustCertsFilePath(String brokerClientTrustCertsFilePath) { + this.brokerClientTrustCertsFilePath = brokerClientTrustCertsFilePath; + return this; + } + public ClusterDataImplBuilder listenerName(String listenerName) { + this.listenerName = listenerName; + return this; + } - public void update(ClusterDataImpl other) { - checkNotNull(other); - this.serviceUrl = other.serviceUrl; - this.serviceUrlTls = other.serviceUrlTls; - this.brokerServiceUrl = other.brokerServiceUrl; - this.brokerServiceUrlTls = other.brokerServiceUrlTls; - this.proxyServiceUrl = other.proxyServiceUrl; - this.proxyProtocol = other.proxyProtocol; - this.authenticationPlugin = other.authenticationPlugin; - this.authenticationParameters = other.authenticationParameters; - this.brokerClientTlsEnabled = other.brokerClientTlsEnabled; - this.tlsAllowInsecureConnection = other.tlsAllowInsecureConnection; - this.brokerClientTlsEnabledWithKeyStore = other.brokerClientTlsEnabledWithKeyStore; - this.brokerClientTlsTrustStoreType = other.brokerClientTlsTrustStoreType; - this.brokerClientTlsTrustStore = other.brokerClientTlsTrustStore; - this.brokerClientTlsTrustStorePassword = other.brokerClientTlsTrustStorePassword; - this.brokerClientTrustCertsFilePath = other.brokerClientTrustCertsFilePath; - this.listenerName = other.listenerName; + public ClusterDataImpl build() { + return new ClusterDataImpl(serviceUrl, serviceUrlTls, brokerServiceUrl, brokerServiceUrlTls, proxyServiceUrl, authenticationPlugin, authenticationParameters, proxyProtocol, peerClusterNames, brokerClientTlsEnabled, tlsAllowInsecureConnection, brokerClientTlsEnabledWithKeyStore, brokerClientTlsTrustStoreType, brokerClientTlsTrustStore, brokerClientTlsTrustStorePassword, brokerClientTrustCertsFilePath, listenerName); + } } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FailureDomainImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FailureDomainImpl.java index d486d228535e1..91e202f8074e4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FailureDomainImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FailureDomainImpl.java @@ -21,11 +21,10 @@ import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; import java.util.HashSet; -import java.util.Objects; import java.util.Set; - +import lombok.AllArgsConstructor; import lombok.Data; -import lombok.ToString; +import lombok.NoArgsConstructor; /** * The data of a failure domain configuration in a cluster. @@ -34,39 +33,32 @@ value = "FailureDomain", description = "The data of a failure domain configuration in a cluster" ) -@ToString @Data -public class FailureDomainImpl implements FailureDomain { +@NoArgsConstructor +@AllArgsConstructor +public final class FailureDomainImpl implements FailureDomain { @ApiModelProperty( name = "brokers", value = "The collection of brokers in the same failure domain", example = "[ 'broker-1', 'broker-2' ]" ) - public Set brokers = new HashSet(); + public Set brokers; - @Override - public Set getBrokers() { - return brokers; + public static FailureDomainImplBuilder builder() { + return new FailureDomainImplBuilder(); } - @Override - public void setBrokers(Set brokers) { - this.brokers = brokers; - } + public static class FailureDomainImplBuilder implements FailureDomain.Builder { + private Set brokers = new HashSet<>(); - @Override - public int hashCode() { - return Objects.hashCode(brokers); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof FailureDomainImpl) { - FailureDomainImpl other = (FailureDomainImpl) obj; - return Objects.equals(brokers, other.brokers); + public FailureDomainImplBuilder brokers(Set brokers) { + this.brokers = brokers; + return this; } - return false; + public FailureDomainImpl build() { + return new FailureDomainImpl(brokers); + } } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataImpl.java index f1ed757b96337..b39bcebe4a13e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionInstanceStatsDataImpl.java @@ -36,7 +36,7 @@ "userExceptionsTotal", "avgProcessLatency", "1min", "lastInvocation", "userMetrics" }) public class FunctionInstanceStatsDataImpl extends FunctionInstanceStatsDataBaseImpl implements FunctionInstanceStatsData { @JsonProperty("1min") - public FunctionInstanceStatsDataBase oneMin = new FunctionInstanceStatsDataBaseImpl(); + public FunctionInstanceStatsDataBaseImpl oneMin = new FunctionInstanceStatsDataBaseImpl(); /** * Timestamp of when the function was last invoked for instance. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionStatsImpl.java index 099622d76c632..91f74c4bb1a3e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/FunctionStatsImpl.java @@ -63,7 +63,7 @@ public class FunctionStatsImpl implements FunctionStats { public Double avgProcessLatency; @JsonProperty("1min") - public FunctionInstanceStatsDataBase oneMin = + public FunctionInstanceStatsDataBaseImpl oneMin = new FunctionInstanceStatsDataBaseImpl(); /** @@ -71,10 +71,9 @@ public class FunctionStatsImpl implements FunctionStats { **/ public Long lastInvocation; - public List instances = new LinkedList<>(); + public List instances = new LinkedList<>(); - @Override - public void addInstance(FunctionInstanceStats functionInstanceStats) { + public void addInstance(FunctionInstanceStatsImpl functionInstanceStats) { instances.add(functionInstanceStats); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java index 1c3ce58e78eb0..5ca9d07c56558 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java @@ -18,17 +18,21 @@ */ package org.apache.pulsar.common.policies.data; -import static com.google.common.base.Preconditions.checkArgument; -import com.google.common.base.Objects; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.Value; +import lombok.extern.jackson.Jacksonized; +import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; import java.util.List; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; - -import lombok.Data; -import org.apache.commons.lang3.StringUtils; +import static com.google.common.base.Preconditions.checkArgument; /** * The data of namespace isolation configuration. @@ -38,23 +42,28 @@ description = "The data of namespace isolation configuration" ) @Data +@AllArgsConstructor +@NoArgsConstructor public class NamespaceIsolationDataImpl implements NamespaceIsolationData { @ApiModelProperty( name = "namespaces", value = "The list of namespaces to apply this namespace isolation data" ) - public List namespaces = new ArrayList(); + private List namespaces; + @ApiModelProperty( name = "primary", value = "The list of primary brokers for serving the list of namespaces in this isolation policy" ) - public List primary = new ArrayList(); + private List primary; + @ApiModelProperty( name = "primary", value = "The list of secondary brokers for serving the list of namespaces in this isolation policy" ) - public List secondary = new ArrayList(); + private List secondary; + @ApiModelProperty( name = "auto_failover_policy", value = "The data of auto-failover policy configuration", @@ -66,31 +75,18 @@ public class NamespaceIsolationDataImpl implements NamespaceIsolationData { + " }" + "}" ) - @SuppressWarnings("checkstyle:MemberName") - public AutoFailoverPolicyDataImpl auto_failover_policy; - - @Override - public int hashCode() { - return Objects.hashCode(namespaces, primary, secondary, - auto_failover_policy); - } + @JsonProperty("auto_failover_policy") + private AutoFailoverPolicyData autoFailoverPolicy; - @Override - public boolean equals(Object obj) { - if (obj instanceof NamespaceIsolationDataImpl) { - NamespaceIsolationDataImpl other = (NamespaceIsolationDataImpl) obj; - return Objects.equal(namespaces, other.namespaces) && Objects.equal(primary, other.primary) - && Objects.equal(secondary, other.secondary) - && Objects.equal(auto_failover_policy, other.auto_failover_policy); - } - - return false; + public static NamespaceIsolationDataImplBuilder builder() { + return new NamespaceIsolationDataImplBuilder(); } public void validate() { checkArgument(namespaces != null && !namespaces.isEmpty() && primary != null && !primary.isEmpty() - && validateRegex(primary) && secondary != null && validateRegex(secondary) && auto_failover_policy != null); - auto_failover_policy.validate(); + && validateRegex(primary) && secondary != null && validateRegex(secondary) + && autoFailoverPolicy != null); + autoFailoverPolicy.validate(); } private boolean validateRegex(List policies) { @@ -108,14 +104,34 @@ private boolean validateRegex(List policies) { return true; } - @Override - public String toString() { - return String.format("namespaces=%s primary=%s secondary=%s auto_failover_policy=%s", namespaces, primary, - secondary, auto_failover_policy); - } + public static class NamespaceIsolationDataImplBuilder implements NamespaceIsolationData.Builder { + private List namespaces = new ArrayList<>(); + private List primary = new ArrayList<>(); + private List secondary = new ArrayList<>(); + private AutoFailoverPolicyData autoFailoverPolicy; + + public NamespaceIsolationDataImplBuilder namespaces(List namespaces) { + this.namespaces = namespaces; + return this; + } + + public NamespaceIsolationDataImplBuilder primary(List primary) { + this.primary = primary; + return this; + } + + public NamespaceIsolationDataImplBuilder secondary(List secondary) { + this.secondary = secondary; + return this; + } - @Override - public void setAuto_failover_policy(AutoFailoverPolicyData autoFailoverPolicyData) { - this.auto_failover_policy = (AutoFailoverPolicyDataImpl) autoFailoverPolicyData; + public NamespaceIsolationDataImplBuilder autoFailoverPolicy(AutoFailoverPolicyData autoFailoverPolicy) { + this.autoFailoverPolicy = autoFailoverPolicy; + return this; + } + + public NamespaceIsolationDataImpl build() { + return new NamespaceIsolationDataImpl(namespaces, primary, secondary, autoFailoverPolicy); + } } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java index d2478e2963d6f..f7f4477eb61f2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java @@ -18,11 +18,12 @@ */ package org.apache.pulsar.common.policies.data; -import static org.apache.pulsar.common.util.FieldParser.value; - import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import java.io.Serializable; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; @@ -33,18 +34,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.Properties; - -import lombok.Data; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; +import static org.apache.pulsar.common.util.FieldParser.value; /** * Definition of the offload policies. */ @Slf4j @Data +@NoArgsConstructor public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { private final static long serialVersionUID = 0L; @@ -181,43 +179,44 @@ public static OffloadPoliciesImpl create(String driver, String region, String bu Integer maxBlockSizeInBytes, Integer readBufferSizeInBytes, Long offloadThresholdInBytes, Long offloadDeletionLagInMillis, OffloadedReadPriority readPriority) { - OffloadPoliciesImpl offloadPolicies = new OffloadPoliciesImpl(); - offloadPolicies.setManagedLedgerOffloadDriver(driver); - offloadPolicies.setManagedLedgerOffloadThresholdInBytes(offloadThresholdInBytes); - offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(offloadDeletionLagInMillis); - - offloadPolicies.setManagedLedgerOffloadBucket(bucket); - offloadPolicies.setManagedLedgerOffloadRegion(region); - offloadPolicies.setManagedLedgerOffloadServiceEndpoint(endpoint); - offloadPolicies.setManagedLedgerOffloadMaxBlockSizeInBytes(maxBlockSizeInBytes); - offloadPolicies.setManagedLedgerOffloadReadBufferSizeInBytes(readBufferSizeInBytes); - offloadPolicies.setManagedLedgerOffloadedReadPriority(readPriority); + OffloadPoliciesImplBuilder builder = builder() + .managedLedgerOffloadDriver(driver) + .managedLedgerOffloadThresholdInBytes(offloadThresholdInBytes) + .managedLedgerOffloadDeletionLagInMillis(offloadDeletionLagInMillis) + .managedLedgerOffloadBucket(bucket) + .managedLedgerOffloadRegion(region) + .managedLedgerOffloadServiceEndpoint(endpoint) + .managedLedgerOffloadMaxBlockSizeInBytes(maxBlockSizeInBytes) + .managedLedgerOffloadReadBufferSizeInBytes(readBufferSizeInBytes) + .managedLedgerOffloadedReadPriority(readPriority); if (driver.equalsIgnoreCase(DRIVER_NAMES.get(0)) || driver.equalsIgnoreCase(DRIVER_NAMES.get(1))) { if (role != null) { - offloadPolicies.setS3ManagedLedgerOffloadRole(role); + builder.s3ManagedLedgerOffloadRole(role); } if (roleSessionName != null) { - offloadPolicies.setS3ManagedLedgerOffloadRoleSessionName(roleSessionName); + builder.s3ManagedLedgerOffloadRoleSessionName(roleSessionName); } if (credentialId != null) { - offloadPolicies.setS3ManagedLedgerOffloadCredentialId(credentialId); + builder.s3ManagedLedgerOffloadCredentialId(credentialId); } if (credentialSecret != null) { - offloadPolicies.setS3ManagedLedgerOffloadCredentialSecret(credentialSecret); + builder.s3ManagedLedgerOffloadCredentialSecret(credentialSecret); } - offloadPolicies.setS3ManagedLedgerOffloadRegion(region); - offloadPolicies.setS3ManagedLedgerOffloadBucket(bucket); - offloadPolicies.setS3ManagedLedgerOffloadServiceEndpoint(endpoint); - offloadPolicies.setS3ManagedLedgerOffloadMaxBlockSizeInBytes(maxBlockSizeInBytes); - offloadPolicies.setS3ManagedLedgerOffloadReadBufferSizeInBytes(readBufferSizeInBytes); + + builder.s3ManagedLedgerOffloadRegion(region) + .s3ManagedLedgerOffloadBucket(bucket) + .s3ManagedLedgerOffloadServiceEndpoint(endpoint) + .s3ManagedLedgerOffloadMaxBlockSizeInBytes(maxBlockSizeInBytes) + .s3ManagedLedgerOffloadReadBufferSizeInBytes(readBufferSizeInBytes); } else if (driver.equalsIgnoreCase(DRIVER_NAMES.get(2))) { - offloadPolicies.setGcsManagedLedgerOffloadRegion(region); - offloadPolicies.setGcsManagedLedgerOffloadBucket(bucket); - offloadPolicies.setGcsManagedLedgerOffloadMaxBlockSizeInBytes(maxBlockSizeInBytes); - offloadPolicies.setGcsManagedLedgerOffloadReadBufferSizeInBytes(readBufferSizeInBytes); + builder.gcsManagedLedgerOffloadRegion(region) + .gcsManagedLedgerOffloadBucket(bucket) + .gcsManagedLedgerOffloadMaxBlockSizeInBytes(maxBlockSizeInBytes) + .gcsManagedLedgerOffloadReadBufferSizeInBytes(readBufferSizeInBytes); } - return offloadPolicies; + + return builder.build(); } public static OffloadPoliciesImpl create(Properties properties) { @@ -239,6 +238,10 @@ public static OffloadPoliciesImpl create(Properties properties) { return data; } + public static OffloadPoliciesImplBuilder builder() { + return new OffloadPoliciesImplBuilder(); + } + public void compatibleWithBrokerConfigFile(Properties properties) { if (!properties.containsKey("managedLedgerOffloadThresholdInBytes") && properties.containsKey(OFFLOAD_THRESHOLD_NAME_IN_CONF_FILE)) { @@ -300,115 +303,6 @@ public boolean bucketValid() { return false; } - @Override - public int hashCode() { - return Objects.hash( - managedLedgerOffloadedReadPriority, - managedLedgerOffloadDriver, - managedLedgerOffloadMaxThreads, - managedLedgerOffloadPrefetchRounds, - managedLedgerOffloadThresholdInBytes, - managedLedgerOffloadDeletionLagInMillis, - s3ManagedLedgerOffloadRegion, - s3ManagedLedgerOffloadBucket, - s3ManagedLedgerOffloadServiceEndpoint, - s3ManagedLedgerOffloadMaxBlockSizeInBytes, - s3ManagedLedgerOffloadReadBufferSizeInBytes, - s3ManagedLedgerOffloadRole, - s3ManagedLedgerOffloadRoleSessionName, - gcsManagedLedgerOffloadRegion, - gcsManagedLedgerOffloadBucket, - gcsManagedLedgerOffloadMaxBlockSizeInBytes, - gcsManagedLedgerOffloadReadBufferSizeInBytes, - gcsManagedLedgerOffloadServiceAccountKeyFile, - fileSystemProfilePath, - fileSystemURI, - managedLedgerOffloadBucket, - managedLedgerOffloadRegion, - managedLedgerOffloadServiceEndpoint, - managedLedgerOffloadMaxBlockSizeInBytes, - managedLedgerOffloadReadBufferSizeInBytes); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - OffloadPoliciesImpl other = (OffloadPoliciesImpl) obj; - return Objects.equals(managedLedgerOffloadedReadPriority, other.getManagedLedgerOffloadedReadPriority()) - && Objects.equals(managedLedgerOffloadDriver, other.getManagedLedgerOffloadDriver()) - && Objects.equals(managedLedgerOffloadMaxThreads, other.getManagedLedgerOffloadMaxThreads()) - && Objects.equals(managedLedgerOffloadPrefetchRounds, other.getManagedLedgerOffloadPrefetchRounds()) - && Objects.equals(managedLedgerOffloadThresholdInBytes, - other.getManagedLedgerOffloadThresholdInBytes()) - && Objects.equals(managedLedgerOffloadDeletionLagInMillis, - other.getManagedLedgerOffloadDeletionLagInMillis()) - && Objects.equals(s3ManagedLedgerOffloadRegion, other.getS3ManagedLedgerOffloadRegion()) - && Objects.equals(s3ManagedLedgerOffloadBucket, other.getS3ManagedLedgerOffloadBucket()) - && Objects.equals(s3ManagedLedgerOffloadServiceEndpoint, - other.getS3ManagedLedgerOffloadServiceEndpoint()) - && Objects.equals(s3ManagedLedgerOffloadMaxBlockSizeInBytes, - other.getS3ManagedLedgerOffloadMaxBlockSizeInBytes()) - && Objects.equals(s3ManagedLedgerOffloadReadBufferSizeInBytes, - other.getS3ManagedLedgerOffloadReadBufferSizeInBytes()) - && Objects.equals(s3ManagedLedgerOffloadRole, other.getS3ManagedLedgerOffloadRole()) - && Objects.equals(s3ManagedLedgerOffloadRoleSessionName, - other.getS3ManagedLedgerOffloadRoleSessionName()) - && Objects.equals(gcsManagedLedgerOffloadRegion, other.getGcsManagedLedgerOffloadRegion()) - && Objects.equals(gcsManagedLedgerOffloadBucket, other.getGcsManagedLedgerOffloadBucket()) - && Objects.equals(gcsManagedLedgerOffloadMaxBlockSizeInBytes, - other.getGcsManagedLedgerOffloadMaxBlockSizeInBytes()) - && Objects.equals(gcsManagedLedgerOffloadReadBufferSizeInBytes, - other.getGcsManagedLedgerOffloadReadBufferSizeInBytes()) - && Objects.equals(gcsManagedLedgerOffloadServiceAccountKeyFile, - other.getGcsManagedLedgerOffloadServiceAccountKeyFile()) - && Objects.equals(fileSystemProfilePath, other.getFileSystemProfilePath()) - && Objects.equals(fileSystemURI, other.getFileSystemURI()) - && Objects.equals(managedLedgerOffloadBucket, other.getManagedLedgerOffloadBucket()) - && Objects.equals(managedLedgerOffloadRegion, other.getManagedLedgerOffloadRegion()) - && Objects.equals(managedLedgerOffloadServiceEndpoint, other.getManagedLedgerOffloadServiceEndpoint()) - && Objects.equals(managedLedgerOffloadMaxBlockSizeInBytes, - other.getManagedLedgerOffloadMaxBlockSizeInBytes()) - && Objects.equals(managedLedgerOffloadReadBufferSizeInBytes, - other.getManagedLedgerOffloadReadBufferSizeInBytes()); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("managedLedgerOffloadedReadPriority", managedLedgerOffloadedReadPriority) - .add("managedLedgerOffloadDriver", managedLedgerOffloadDriver) - .add("managedLedgerOffloadMaxThreads", managedLedgerOffloadMaxThreads) - .add("managedLedgerOffloadPrefetchRounds", managedLedgerOffloadPrefetchRounds) - .add("managedLedgerOffloadAutoTriggerSizeThresholdBytes", - managedLedgerOffloadThresholdInBytes) - .add("managedLedgerOffloadDeletionLagInMillis", managedLedgerOffloadDeletionLagInMillis) - .add("s3ManagedLedgerOffloadRegion", s3ManagedLedgerOffloadRegion) - .add("s3ManagedLedgerOffloadBucket", s3ManagedLedgerOffloadBucket) - .add("s3ManagedLedgerOffloadServiceEndpoint", s3ManagedLedgerOffloadServiceEndpoint) - .add("s3ManagedLedgerOffloadMaxBlockSizeInBytes", s3ManagedLedgerOffloadMaxBlockSizeInBytes) - .add("s3ManagedLedgerOffloadReadBufferSizeInBytes", s3ManagedLedgerOffloadReadBufferSizeInBytes) - .add("s3ManagedLedgerOffloadRole", s3ManagedLedgerOffloadRole) - .add("s3ManagedLedgerOffloadRoleSessionName", s3ManagedLedgerOffloadRoleSessionName) - .add("gcsManagedLedgerOffloadRegion", gcsManagedLedgerOffloadRegion) - .add("gcsManagedLedgerOffloadBucket", gcsManagedLedgerOffloadBucket) - .add("gcsManagedLedgerOffloadMaxBlockSizeInBytes", gcsManagedLedgerOffloadMaxBlockSizeInBytes) - .add("gcsManagedLedgerOffloadReadBufferSizeInBytes", gcsManagedLedgerOffloadReadBufferSizeInBytes) - .add("gcsManagedLedgerOffloadServiceAccountKeyFile", gcsManagedLedgerOffloadServiceAccountKeyFile) - .add("fileSystemProfilePath", fileSystemProfilePath) - .add("fileSystemURI", fileSystemURI) - .add("managedLedgerOffloadBucket", managedLedgerOffloadBucket) - .add("managedLedgerOffloadRegion", managedLedgerOffloadRegion) - .add("managedLedgerOffloadServiceEndpoint", managedLedgerOffloadServiceEndpoint) - .add("managedLedgerOffloadMaxBlockSizeInBytes", managedLedgerOffloadMaxBlockSizeInBytes) - .add("managedLedgerOffloadReadBufferSizeInBytes", managedLedgerOffloadReadBufferSizeInBytes) - .toString(); - } - public Properties toProperties() { Properties properties = new Properties(); setProperty(properties, "managedLedgerOffloadedReadPriority", this.getManagedLedgerOffloadedReadPriority()); @@ -578,4 +472,151 @@ private static Object getCompatibleValue(Properties properties, Field field) { return value((String) object, field); } + public static class OffloadPoliciesImplBuilder { + private OffloadPoliciesImpl impl = new OffloadPoliciesImpl(); + + public OffloadPoliciesImplBuilder offloadersDirectory(String offloadersDirectory) { + impl.offloadersDirectory = offloadersDirectory; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadDriver(String managedLedgerOffloadDriver) { + impl.managedLedgerOffloadDriver = managedLedgerOffloadDriver; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadMaxThreads(Integer managedLedgerOffloadMaxThreads) { + impl.managedLedgerOffloadMaxThreads = managedLedgerOffloadMaxThreads; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadPrefetchRounds(Integer managedLedgerOffloadPrefetchRounds) { + impl.managedLedgerOffloadPrefetchRounds = managedLedgerOffloadPrefetchRounds; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadThresholdInBytes(Long managedLedgerOffloadThresholdInBytes) { + impl.managedLedgerOffloadThresholdInBytes = managedLedgerOffloadThresholdInBytes; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadDeletionLagInMillis(Long managedLedgerOffloadDeletionLagInMillis) { + impl.managedLedgerOffloadDeletionLagInMillis = managedLedgerOffloadDeletionLagInMillis; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadedReadPriority(OffloadedReadPriority managedLedgerOffloadedReadPriority) { + impl.managedLedgerOffloadedReadPriority = managedLedgerOffloadedReadPriority; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadRegion(String s3ManagedLedgerOffloadRegion) { + impl.s3ManagedLedgerOffloadRegion = s3ManagedLedgerOffloadRegion; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadBucket(String s3ManagedLedgerOffloadBucket) { + impl.s3ManagedLedgerOffloadBucket = s3ManagedLedgerOffloadBucket; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadServiceEndpoint(String s3ManagedLedgerOffloadServiceEndpoint) { + impl.s3ManagedLedgerOffloadServiceEndpoint = s3ManagedLedgerOffloadServiceEndpoint; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadMaxBlockSizeInBytes(Integer s3ManagedLedgerOffloadMaxBlockSizeInBytes) { + impl.s3ManagedLedgerOffloadMaxBlockSizeInBytes = s3ManagedLedgerOffloadMaxBlockSizeInBytes; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadReadBufferSizeInBytes(Integer s3ManagedLedgerOffloadReadBufferSizeInBytes) { + impl.s3ManagedLedgerOffloadReadBufferSizeInBytes = s3ManagedLedgerOffloadReadBufferSizeInBytes; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadCredentialId(String s3ManagedLedgerOffloadCredentialId) { + impl.s3ManagedLedgerOffloadCredentialId = s3ManagedLedgerOffloadCredentialId; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadCredentialSecret(String s3ManagedLedgerOffloadCredentialSecret) { + impl.s3ManagedLedgerOffloadCredentialSecret = s3ManagedLedgerOffloadCredentialSecret; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadRole(String s3ManagedLedgerOffloadRole) { + impl.s3ManagedLedgerOffloadRole = s3ManagedLedgerOffloadRole; + return this; + } + + public OffloadPoliciesImplBuilder s3ManagedLedgerOffloadRoleSessionName(String s3ManagedLedgerOffloadRoleSessionName) { + impl.s3ManagedLedgerOffloadRoleSessionName = s3ManagedLedgerOffloadRoleSessionName; + return this; + } + + public OffloadPoliciesImplBuilder gcsManagedLedgerOffloadRegion(String gcsManagedLedgerOffloadRegion) { + impl.gcsManagedLedgerOffloadRegion = gcsManagedLedgerOffloadRegion; + return this; + } + + public OffloadPoliciesImplBuilder gcsManagedLedgerOffloadBucket(String gcsManagedLedgerOffloadBucket) { + impl.gcsManagedLedgerOffloadBucket = gcsManagedLedgerOffloadBucket; + return this; + } + + public OffloadPoliciesImplBuilder gcsManagedLedgerOffloadMaxBlockSizeInBytes(Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes) { + impl.gcsManagedLedgerOffloadMaxBlockSizeInBytes = gcsManagedLedgerOffloadMaxBlockSizeInBytes; + return this; + } + + public OffloadPoliciesImplBuilder gcsManagedLedgerOffloadReadBufferSizeInBytes(Integer gcsManagedLedgerOffloadReadBufferSizeInBytes) { + impl.gcsManagedLedgerOffloadReadBufferSizeInBytes = gcsManagedLedgerOffloadReadBufferSizeInBytes; + return this; + } + + public OffloadPoliciesImplBuilder gcsManagedLedgerOffloadServiceAccountKeyFile(String gcsManagedLedgerOffloadServiceAccountKeyFile) { + impl.gcsManagedLedgerOffloadServiceAccountKeyFile = gcsManagedLedgerOffloadServiceAccountKeyFile; + return this; + } + + public OffloadPoliciesImplBuilder fileSystemProfilePath(String fileSystemProfilePath) { + impl.fileSystemProfilePath = fileSystemProfilePath; + return this; + } + + public OffloadPoliciesImplBuilder fileSystemURI(String fileSystemURI) { + impl.fileSystemURI = fileSystemURI; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadBucket(String managedLedgerOffloadBucket) { + impl.managedLedgerOffloadBucket = managedLedgerOffloadBucket; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadRegion(String managedLedgerOffloadRegion) { + impl.managedLedgerOffloadRegion = managedLedgerOffloadRegion; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadServiceEndpoint(String managedLedgerOffloadServiceEndpoint) { + impl.managedLedgerOffloadServiceEndpoint = managedLedgerOffloadServiceEndpoint; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadMaxBlockSizeInBytes(Integer managedLedgerOffloadMaxBlockSizeInBytes) { + impl.managedLedgerOffloadMaxBlockSizeInBytes = managedLedgerOffloadMaxBlockSizeInBytes; + return this; + } + + public OffloadPoliciesImplBuilder managedLedgerOffloadReadBufferSizeInBytes(Integer managedLedgerOffloadReadBufferSizeInBytes) { + impl.managedLedgerOffloadReadBufferSizeInBytes = managedLedgerOffloadReadBufferSizeInBytes; + return this; + } + + public OffloadPoliciesImpl build() { + return impl; + } + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PersistentTopicStats.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PersistentTopicStats.java index e8385628b1eea..b8aa83d988a97 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PersistentTopicStats.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PersistentTopicStats.java @@ -21,6 +21,6 @@ /** * Maintain compatibility with previous class name. */ -public class PersistentTopicStats extends TopicStats { +public interface PersistentTopicStats extends TopicStats { } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PoliciesUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PoliciesUtil.java index def6a17388d54..c1d61f41af5fc 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PoliciesUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PoliciesUtil.java @@ -31,12 +31,13 @@ public class PoliciesUtil { public static BundlesData defaultBundle() { - BundlesData bundle = new BundlesData(1); List boundaries = new ArrayList<>(); boundaries.add(FIRST_BOUNDARY); boundaries.add(LAST_BOUNDARY); - bundle.setBoundaries(boundaries); - return bundle; + return BundlesData.builder() + .numBundles(1) + .boundaries(boundaries) + .build(); } public static void setStorageQuota(Policies polices, BacklogQuota quota) { @@ -66,6 +67,9 @@ public static BundlesData getBundles(int numBundles) { } curPartition += segSize; } - return new BundlesData(partitions); + return BundlesData.builder() + .boundaries(partitions) + .numBundles(numBundles) + .build(); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantInfoImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantInfoImpl.java index fc00b86200bd0..3a5167ad393d3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantInfoImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantInfoImpl.java @@ -18,16 +18,20 @@ */ package org.apache.pulsar.common.policies.data; -import com.google.common.collect.Sets; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; +import java.util.HashSet; import java.util.Set; +import lombok.AllArgsConstructor; import lombok.Data; +import lombok.NoArgsConstructor; /** * Information of admin roles and allowed clusters for tenant. */ @Data +@AllArgsConstructor +@NoArgsConstructor @ApiModel(value = "TenantInfo", description = "Information of adminRoles and allowedClusters for tenant") public class TenantInfoImpl implements TenantInfo { /** @@ -48,33 +52,36 @@ public class TenantInfoImpl implements TenantInfo { ) private Set allowedClusters; - public TenantInfoImpl() { - adminRoles = Sets.newHashSet(); - allowedClusters = Sets.newHashSet(); - } - public TenantInfoImpl(Set adminRoles, Set allowedClusters) { - this.adminRoles = adminRoles; - this.allowedClusters = allowedClusters; + public static TenantInfoImplBuilder builder() { + return new TenantInfoImplBuilder(); } - @Override - public Set getAdminRoles() { - return adminRoles; - } + public static class TenantInfoImplBuilder implements TenantInfo.Builder { + private Set adminRoles; + private Set allowedClusters; - @Override - public void setAdminRoles(Set adminRoles) { - this.adminRoles = adminRoles; - } + TenantInfoImplBuilder() { + } - @Override - public Set getAllowedClusters() { - return allowedClusters; - } + public TenantInfoImplBuilder adminRoles(Set adminRoles) { + this.adminRoles = adminRoles; + return this; + } + + public TenantInfoImplBuilder allowedClusters(Set allowedClusters) { + this.allowedClusters = allowedClusters; + return this; + } - @Override - public void setAllowedClusters(Set allowedClusters) { - this.allowedClusters = allowedClusters; + public TenantInfoImpl build() { + if (adminRoles == null) { + adminRoles = new HashSet<>(); + } + if (allowedClusters == null) { + allowedClusters = new HashSet<>(); + } + return new TenantInfoImpl(adminRoles, allowedClusters); + } } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicPolicies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicPolicies.java index 0b03425a9e07a..936ef433c43b6 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicPolicies.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicPolicies.java @@ -30,6 +30,8 @@ import lombok.NoArgsConstructor; import lombok.Setter; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; /** @@ -43,7 +45,7 @@ @Setter public class TopicPolicies { - private Map backLogQuotaMap = Maps.newHashMap(); + private Map backLogQuotaMap = Maps.newHashMap(); private PersistencePolicies persistence = null; private RetentionPolicies retentionPolicies = null; private Boolean deduplicationEnabled = null; @@ -57,15 +59,15 @@ public class TopicPolicies { private Boolean delayedDeliveryEnabled = null; private OffloadPoliciesImpl offloadPolicies; private InactiveTopicPolicies inactiveTopicPolicies = null; - private DispatchRate dispatchRate = null; - private DispatchRate subscriptionDispatchRate = null; + private DispatchRateImpl dispatchRate = null; + private DispatchRateImpl subscriptionDispatchRate = null; private Long compactionThreshold = null; private PublishRate publishRate = null; private SubscribeRate subscribeRate = null; private Integer deduplicationSnapshotIntervalSeconds = null; private Integer maxMessageSize = null; private Integer maxSubscriptionsPerTopic = null; - private DispatchRate replicatorDispatchRate = null; + private DispatchRateImpl replicatorDispatchRate = null; private List subscriptionTypesEnabled = new ArrayList<>(); public boolean isReplicatorDispatchRateSet() { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java new file mode 100644 index 0000000000000..7a87ef88e3c0a --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import lombok.Data; +import org.apache.pulsar.common.policies.data.ConsumerStats; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Consumer statistics. + */ +@Data +public class ConsumerStatsImpl implements ConsumerStats { + /** Total rate of messages delivered to the consumer (msg/s). */ + public double msgRateOut; + + /** Total throughput delivered to the consumer (bytes/s). */ + public double msgThroughputOut; + + /** Total bytes delivered to consumer (bytes). */ + public long bytesOutCounter; + + /** Total messages delivered to consumer (msg). */ + public long msgOutCounter; + + /** Total rate of messages redelivered by this consumer (msg/s). */ + public double msgRateRedeliver; + + /** Total chunked messages dispatched. */ + public double chunkedMessageRate; + + /** Name of the consumer. */ + public String consumerName; + + /** Number of available message permits for the consumer. */ + public int availablePermits; + + /** Number of unacknowledged messages for the consumer. */ + public int unackedMessages; + + /** Number of average messages per entry for the consumer consumed. */ + public int avgMessagesPerEntry; + + /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ + public boolean blockedConsumerOnUnackedMsgs; + + /** The read position of the cursor when the consumer joining. */ + public String readPositionWhenJoining; + + /** Address of this consumer. */ + private int addressOffset = -1; + private int addressLength; + + /** Timestamp of connection. */ + private int connectedSinceOffset = -1; + private int connectedSinceLength; + + /** Client library version. */ + private int clientVersionOffset = -1; + private int clientVersionLength; + + public long lastAckedTimestamp; + public long lastConsumedTimestamp; + + /** Hash ranges assigned to this consumer if is Key_Shared sub mode. **/ + public List keyHashRanges; + + /** Metadata (key/value strings) associated with this consumer. */ + public Map metadata; + + /** + * In order to prevent multiple string object allocation under stats: create a string-buffer + * that stores data for all string place-holders. + */ + private StringBuilder stringBuffer = new StringBuilder(); + + public ConsumerStatsImpl add(ConsumerStatsImpl stats) { + Objects.requireNonNull(stats); + this.msgRateOut += stats.msgRateOut; + this.msgThroughputOut += stats.msgThroughputOut; + this.bytesOutCounter += stats.bytesOutCounter; + this.msgOutCounter += stats.msgOutCounter; + this.msgRateRedeliver += stats.msgRateRedeliver; + this.availablePermits += stats.availablePermits; + this.unackedMessages += stats.unackedMessages; + this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; + this.readPositionWhenJoining = stats.readPositionWhenJoining; + return this; + } + + public String getAddress() { + return addressOffset == -1 ? null : stringBuffer.substring(addressOffset, addressOffset + addressLength); + } + + public void setAddress(String address) { + if (address == null) { + this.addressOffset = -1; + return; + } + this.addressOffset = this.stringBuffer.length(); + this.addressLength = address.length(); + this.stringBuffer.append(address); + } + + public String getConnectedSince() { + return connectedSinceOffset == -1 ? null + : stringBuffer.substring(connectedSinceOffset, connectedSinceOffset + connectedSinceLength); + } + + public void setConnectedSince(String connectedSince) { + if (connectedSince == null) { + this.connectedSinceOffset = -1; + return; + } + this.connectedSinceOffset = this.stringBuffer.length(); + this.connectedSinceLength = connectedSince.length(); + this.stringBuffer.append(connectedSince); + } + + public String getClientVersion() { + return clientVersionOffset == -1 ? null + : stringBuffer.substring(clientVersionOffset, clientVersionOffset + clientVersionLength); + } + + public void setClientVersion(String clientVersion) { + if (clientVersion == null) { + this.clientVersionOffset = -1; + return; + } + this.clientVersionOffset = this.stringBuffer.length(); + this.clientVersionLength = clientVersion.length(); + this.stringBuffer.append(clientVersion); + } + + public String getReadPositionWhenJoining() { + return readPositionWhenJoining; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java new file mode 100644 index 0000000000000..c5100376c87bd --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; +import java.util.Objects; + +/** + * Non-persistent publisher statistics. + */ +@SuppressFBWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +public class NonPersistentPublisherStatsImpl extends PublisherStatsImpl implements NonPersistentPublisherStats { + /** + * for non-persistent topic: broker drops msg if publisher publishes messages more than configured max inflight + * messages per connection. + **/ + @Getter + public double msgDropRate; + + public NonPersistentPublisherStatsImpl add(NonPersistentPublisherStatsImpl stats) { + Objects.requireNonNull(stats); + super.add(stats); + this.msgDropRate += stats.msgDropRate; + return this; + } +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java new file mode 100644 index 0000000000000..3c6c4809d5326 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.Objects; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats; + +/** + * Statistics for a non-persistent replicator. + */ +@SuppressFBWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +public class NonPersistentReplicatorStatsImpl extends ReplicatorStatsImpl implements NonPersistentReplicatorStats { + + /** + * for non-persistent topic: broker drops msg for replicator if replicator connection is not writable. + **/ + @Getter + public double msgDropRate; + + public NonPersistentReplicatorStatsImpl add(NonPersistentReplicatorStatsImpl stats) { + Objects.requireNonNull(stats); + super.add(stats); + this.msgDropRate += stats.msgDropRate; + return this; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentSubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentSubscriptionStatsImpl.java new file mode 100644 index 0000000000000..7e5fb12b06449 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentSubscriptionStatsImpl.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.common.policies.data.stats; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.Objects; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats; + +/** + * Statistics for subscription to non-persistent topics. + */ +@SuppressFBWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +public class NonPersistentSubscriptionStatsImpl extends SubscriptionStatsImpl implements NonPersistentSubscriptionStats { + + /** + * for non-persistent topic: broker drops msg for subscription if none of the consumer available for message + * delivery. + **/ + @Getter + public double msgDropRate; + + public void reset() { + super.reset(); + msgDropRate = 0; + } + + // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current + // stats + public NonPersistentSubscriptionStatsImpl add(NonPersistentSubscriptionStatsImpl stats) { + Objects.requireNonNull(stats); + super.add(stats); + this.msgDropRate += stats.msgDropRate; + return this; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java new file mode 100644 index 0000000000000..06f59cb13e6e9 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import lombok.Data; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; +import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats; +import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats; +import org.apache.pulsar.common.policies.data.NonPersistentTopicStats; +import org.apache.pulsar.common.policies.data.PublisherStats; +import org.apache.pulsar.common.policies.data.ReplicatorStats; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +/** + * Statistics for a non-persistent topic. + */ +@SuppressFBWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +public class NonPersistentTopicStatsImpl extends TopicStatsImpl implements NonPersistentTopicStats { + + /** + * for non-persistent topic: broker drops msg if publisher publishes messages more than configured max inflight + * messages per connection. + **/ + @Getter + public double msgDropRate; + + /** List of connected publishers on this topic w/ their stats. */ + @Getter + public List publishers; + + /** Map of subscriptions with their individual statistics. */ + @Getter + public Map subscriptions; + + /** Map of replication statistics by remote cluster context. */ + @Getter + public Map replication; + + @SuppressFBWarnings(value = "MF_CLASS_MASKS_FIELD", justification = "expected to override") + public List getPublishers() { + return (List) publishers; + } + + @SuppressFBWarnings(value = "MF_CLASS_MASKS_FIELD", justification = "expected to override") + public Map getSubscriptions() { + return (Map) subscriptions; + } + + @SuppressFBWarnings(value = "MF_CLASS_MASKS_FIELD", justification = "expected to override") + public Map getReplication() { + return (Map) replication; + } + + @Override + public double getMsgDropRate() { + return msgDropRate; + } + + public NonPersistentTopicStatsImpl() { + this.publishers = new ArrayList<>(); + this.subscriptions = new HashMap<>(); + this.replication = new TreeMap<>(); + } + + public void reset() { + super.reset(); + this.msgDropRate = 0; + } + + // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current + // stats. + public NonPersistentTopicStatsImpl add(NonPersistentTopicStatsImpl stats) { + Objects.requireNonNull(stats); + super.add(stats); + this.msgDropRate += stats.msgDropRate; + return this; + } + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PartitionedTopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PartitionedTopicStatsImpl.java new file mode 100644 index 0000000000000..0b07dac85c3aa --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PartitionedTopicStatsImpl.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.HashMap; +import java.util.Map; +import lombok.Getter; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.TopicStats; + +/** + * Statistics for a partitioned topic. + */ +@SuppressFBWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +public class PartitionedTopicStatsImpl extends TopicStatsImpl implements PartitionedTopicStats { + + @Getter + public PartitionedTopicMetadata metadata; + + @Getter + public Map partitions; + + public PartitionedTopicStatsImpl() { + super(); + metadata = new PartitionedTopicMetadata(); + partitions = new HashMap<>(); + } + + public PartitionedTopicStatsImpl(PartitionedTopicMetadata metadata) { + this(); + this.metadata = metadata; + } + + @Override + public void reset() { + super.reset(); + partitions.clear(); + metadata.partitions = 0; + } + +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java new file mode 100644 index 0000000000000..683b1eac0ab9f --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java @@ -0,0 +1,145 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import lombok.Data; +import org.apache.pulsar.client.api.ProducerAccessMode; +import org.apache.pulsar.common.policies.data.PublisherStats; +import java.util.Map; + +/** + * Statistics about a publisher. + */ +@Data +public class PublisherStatsImpl implements PublisherStats { + private int count; + + public ProducerAccessMode accessMode; + + /** Total rate of messages published by this publisher (msg/s). */ + public double msgRateIn; + + /** Total throughput of messages published by this publisher (byte/s). */ + public double msgThroughputIn; + + /** Average message size published by this publisher. */ + public double averageMsgSize; + + /** total chunked message count received. **/ + public double chunkedMessageRate; + + /** Id of this publisher. */ + public long producerId; + + /** Producer name. */ + private int producerNameOffset = -1; + private int producerNameLength; + + /** Address of this publisher. */ + private int addressOffset = -1; + private int addressLength; + + /** Timestamp of connection. */ + private int connectedSinceOffset = -1; + private int connectedSinceLength; + + /** Client library version. */ + private int clientVersionOffset = -1; + private int clientVersionLength; + + /** + * In order to prevent multiple string objects under stats: create a string-buffer that stores data for all string + * place-holders. + */ + private StringBuilder stringBuffer = new StringBuilder(); + + /** Metadata (key/value strings) associated with this publisher. */ + public Map metadata; + + public PublisherStatsImpl add(PublisherStatsImpl stats) { + if (stats == null) { + throw new NullPointerException(); + } + this.count++; + this.msgRateIn += stats.msgRateIn; + this.msgThroughputIn += stats.msgThroughputIn; + double newAverageMsgSize = (this.averageMsgSize * (this.count - 1) + stats.averageMsgSize) / this.count; + this.averageMsgSize = newAverageMsgSize; + return this; + } + + public String getProducerName() { + return producerNameOffset == -1 ? null + : stringBuffer.substring(producerNameOffset, producerNameOffset + producerNameLength); + } + + public void setProducerName(String producerName) { + if (producerName == null) { + this.producerNameOffset = -1; + return; + } + this.producerNameOffset = this.stringBuffer.length(); + this.producerNameLength = producerName.length(); + this.stringBuffer.append(producerName); + } + + public String getAddress() { + return addressOffset == -1 ? null : stringBuffer.substring(addressOffset, addressOffset + addressLength); + } + + public void setAddress(String address) { + if (address == null) { + this.addressOffset = -1; + return; + } + this.addressOffset = this.stringBuffer.length(); + this.addressLength = address.length(); + this.stringBuffer.append(address); + } + + public String getConnectedSince() { + return connectedSinceOffset == -1 ? null + : stringBuffer.substring(connectedSinceOffset, connectedSinceOffset + connectedSinceLength); + } + + public void setConnectedSince(String connectedSince) { + if (connectedSince == null) { + this.connectedSinceOffset = -1; + return; + } + this.connectedSinceOffset = this.stringBuffer.length(); + this.connectedSinceLength = connectedSince.length(); + this.stringBuffer.append(connectedSince); + } + + public String getClientVersion() { + return clientVersionOffset == -1 ? null + : stringBuffer.substring(clientVersionOffset, clientVersionOffset + clientVersionLength); + } + + public void setClientVersion(String clientVersion) { + if (clientVersion == null) { + this.clientVersionOffset = -1; + return; + } + this.clientVersionOffset = this.stringBuffer.length(); + this.clientVersionLength = clientVersion.length(); + this.stringBuffer.append(clientVersion); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java new file mode 100644 index 0000000000000..0ca18e19ee228 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import lombok.Data; +import org.apache.pulsar.common.policies.data.ReplicatorStats; +import java.util.Objects; + +/** + * Statistics about a replicator. + */ +@Data +public class ReplicatorStatsImpl implements ReplicatorStats { + + /** Total rate of messages received from the remote cluster (msg/s). */ + public double msgRateIn; + + /** Total throughput received from the remote cluster (bytes/s). */ + public double msgThroughputIn; + + /** Total rate of messages delivered to the replication-subscriber (msg/s). */ + public double msgRateOut; + + /** Total throughput delivered to the replication-subscriber (bytes/s). */ + public double msgThroughputOut; + + /** Total rate of messages expired (msg/s). */ + public double msgRateExpired; + + /** Number of messages pending to be replicated to remote cluster. */ + public long replicationBacklog; + + /** is the replication-subscriber up and running to replicate to remote cluster. */ + public boolean connected; + + /** Time in seconds from the time a message was produced to the time when it is about to be replicated. */ + public long replicationDelayInSeconds; + + /** Address of incoming replication connection. */ + public String inboundConnection; + + /** Timestamp of incoming connection establishment time. */ + public String inboundConnectedSince; + + /** Address of outbound replication connection. */ + public String outboundConnection; + + /** Timestamp of outbound connection establishment time. */ + public String outboundConnectedSince; + + public ReplicatorStatsImpl add(ReplicatorStatsImpl stats) { + Objects.requireNonNull(stats); + this.msgRateIn += stats.msgRateIn; + this.msgThroughputIn += stats.msgThroughputIn; + this.msgRateOut += stats.msgRateOut; + this.msgThroughputOut += stats.msgThroughputOut; + this.msgRateExpired += stats.msgRateExpired; + this.replicationBacklog += stats.replicationBacklog; + this.connected &= stats.connected; + this.replicationDelayInSeconds = Math.max(this.replicationDelayInSeconds, stats.replicationDelayInSeconds); + return this; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java new file mode 100644 index 0000000000000..03e4681883f54 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -0,0 +1,173 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import lombok.Data; +import org.apache.pulsar.common.policies.data.SubscriptionStats; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Statistics about subscription. + */ +@Data +public class SubscriptionStatsImpl implements SubscriptionStats { + /** Total rate of messages delivered on this subscription (msg/s). */ + public double msgRateOut; + + /** Total throughput delivered on this subscription (bytes/s). */ + public double msgThroughputOut; + + /** Total bytes delivered to consumer (bytes). */ + public long bytesOutCounter; + + /** Total messages delivered to consumer (msg). */ + public long msgOutCounter; + + /** Total rate of messages redelivered on this subscription (msg/s). */ + public double msgRateRedeliver; + + /** Chunked message dispatch rate. */ + public int chunkedMessageRate; + + /** Number of messages in the subscription backlog. */ + public long msgBacklog; + + /** Size of backlog in byte. **/ + public long backlogSize; + + /** Number of messages in the subscription backlog that do not contain the delay messages. */ + public long msgBacklogNoDelayed; + + /** Flag to verify if subscription is blocked due to reaching threshold of unacked messages. */ + public boolean blockedSubscriptionOnUnackedMsgs; + + /** Number of delayed messages currently being tracked. */ + public long msgDelayed; + + /** Number of unacknowledged messages for the subscription. */ + public long unackedMessages; + + /** Whether this subscription is Exclusive or Shared or Failover. */ + public String type; + + /** The name of the consumer that is active for single active consumer subscriptions i.e. failover or exclusive. */ + public String activeConsumerName; + + /** Total rate of messages expired on this subscription (msg/s). */ + public double msgRateExpired; + + /** Total messages expired on this subscription. */ + public long totalMsgExpired; + + /** Last message expire execution timestamp. */ + public long lastExpireTimestamp; + + /** Last received consume flow command timestamp. */ + public long lastConsumedFlowTimestamp; + + /** Last consume message timestamp. */ + public long lastConsumedTimestamp; + + /** Last acked message timestamp. */ + public long lastAckedTimestamp; + + /** Last MarkDelete position advanced timesetamp. */ + public long lastMarkDeleteAdvancedTimestamp; + + /** List of connected consumers on this subscription w/ their stats. */ + public List consumers; + + /** Tells whether this subscription is durable or ephemeral (eg.: from a reader). */ + public boolean isDurable; + + /** Mark that the subscription state is kept in sync across different regions. */ + public boolean isReplicated; + + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + public Map consumersAfterMarkDeletePosition; + + /** The number of non-contiguous deleted messages ranges. */ + public int nonContiguousDeletedMessagesRanges; + + /** The serialized size of non-contiguous deleted messages ranges. */ + public int nonContiguousDeletedMessagesRangesSerializedSize; + + public SubscriptionStatsImpl() { + this.consumers = new ArrayList<>(); + this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); + } + + public void reset() { + msgRateOut = 0; + msgThroughputOut = 0; + bytesOutCounter = 0; + msgOutCounter = 0; + msgRateRedeliver = 0; + msgBacklog = 0; + backlogSize = 0; + msgBacklogNoDelayed = 0; + unackedMessages = 0; + msgRateExpired = 0; + totalMsgExpired = 0; + lastExpireTimestamp = 0L; + lastMarkDeleteAdvancedTimestamp = 0L; + consumers.clear(); + consumersAfterMarkDeletePosition.clear(); + nonContiguousDeletedMessagesRanges = 0; + nonContiguousDeletedMessagesRangesSerializedSize = 0; + } + + // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current + // stats + public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { + Objects.requireNonNull(stats); + this.msgRateOut += stats.msgRateOut; + this.msgThroughputOut += stats.msgThroughputOut; + this.bytesOutCounter += stats.bytesOutCounter; + this.msgOutCounter += stats.msgOutCounter; + this.msgRateRedeliver += stats.msgRateRedeliver; + this.msgBacklog += stats.msgBacklog; + this.backlogSize += stats.backlogSize; + this.msgBacklogNoDelayed += stats.msgBacklogNoDelayed; + this.msgDelayed += stats.msgDelayed; + this.unackedMessages += stats.unackedMessages; + this.msgRateExpired += stats.msgRateExpired; + this.totalMsgExpired += stats.totalMsgExpired; + this.isReplicated |= stats.isReplicated; + this.isDurable |= stats.isDurable; + if (this.consumers.size() != stats.consumers.size()) { + for (int i = 0; i < stats.consumers.size(); i++) { + ConsumerStatsImpl consumerStats = new ConsumerStatsImpl(); + this.consumers.add(consumerStats.add(stats.consumers.get(i))); + } + } else { + for (int i = 0; i < stats.consumers.size(); i++) { + this.consumers.get(i).add(stats.consumers.get(i)); + } + } + this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); + this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; + this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; + return this; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java new file mode 100644 index 0000000000000..941c318b70195 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -0,0 +1,213 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data.stats; + +import lombok.AccessLevel; +import lombok.Data; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.PublisherStats; +import org.apache.pulsar.common.policies.data.ReplicatorStats; +import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.apache.pulsar.common.policies.data.TopicStats; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +/** + * Statistics for a Pulsar topic. + */ +@Data +public class TopicStatsImpl implements TopicStats { + private int count; + + /** Total rate of messages published on the topic (msg/s). */ + public double msgRateIn; + + /** Total throughput of messages published on the topic (byte/s). */ + public double msgThroughputIn; + + /** Total rate of messages dispatched for the topic (msg/s). */ + public double msgRateOut; + + /** Total throughput of messages dispatched for the topic (byte/s). */ + public double msgThroughputOut; + + /** Total bytes published to the topic (bytes). */ + public long bytesInCounter; + + /** Total messages published to the topic (msg). */ + public long msgInCounter; + + /** Total bytes delivered to consumer (bytes). */ + public long bytesOutCounter; + + /** Total messages delivered to consumer (msg). */ + public long msgOutCounter; + + /** Average size of published messages (bytes). */ + public double averageMsgSize; + + /** Topic has chunked message published on it. */ + public boolean msgChunkPublished; + + /** Space used to store the messages for the topic (bytes). */ + public long storageSize; + + /** Get estimated total unconsumed or backlog size in bytes. */ + public long backlogSize; + + /** Space used to store the offloaded messages for the topic/. */ + public long offloadedStorageSize; + + /** List of connected publishers on this topic w/ their stats. */ + @Getter(AccessLevel.NONE) + public List publishers; + + public int waitingPublishers; + + /** Map of subscriptions with their individual statistics. */ + @Getter(AccessLevel.NONE) + public Map subscriptions; + + /** Map of replication statistics by remote cluster context. */ + @Getter(AccessLevel.NONE) + public Map replication; + + public String deduplicationStatus; + + /** The topic epoch or empty if not set. */ + public Long topicEpoch; + + /** The number of non-contiguous deleted messages ranges. */ + public int nonContiguousDeletedMessagesRanges; + + /** The serialized size of non-contiguous deleted messages ranges. */ + public int nonContiguousDeletedMessagesRangesSerializedSize; + + public List getPublishers() { + return publishers; + } + + public Map getSubscriptions() { + return subscriptions; + } + + public Map getReplication() { + return replication; + } + + public TopicStatsImpl() { + this.publishers = new ArrayList<>(); + this.subscriptions = new HashMap<>(); + this.replication = new TreeMap<>(); + } + + public void reset() { + this.count = 0; + this.msgRateIn = 0; + this.msgThroughputIn = 0; + this.msgRateOut = 0; + this.msgThroughputOut = 0; + this.averageMsgSize = 0; + this.storageSize = 0; + this.backlogSize = 0; + this.bytesInCounter = 0; + this.msgInCounter = 0; + this.bytesOutCounter = 0; + this.msgOutCounter = 0; + this.publishers.clear(); + this.subscriptions.clear(); + this.waitingPublishers = 0; + this.replication.clear(); + this.deduplicationStatus = null; + this.topicEpoch = null; + this.nonContiguousDeletedMessagesRanges = 0; + this.nonContiguousDeletedMessagesRangesSerializedSize = 0; + this.offloadedStorageSize = 0; + } + + // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current + // stats. + public TopicStatsImpl add(TopicStats ts) { + TopicStatsImpl stats = (TopicStatsImpl) ts; + + this.count++; + this.msgRateIn += stats.msgRateIn; + this.msgThroughputIn += stats.msgThroughputIn; + this.msgRateOut += stats.msgRateOut; + this.msgThroughputOut += stats.msgThroughputOut; + this.bytesInCounter += stats.bytesInCounter; + this.msgInCounter += stats.msgInCounter; + this.bytesOutCounter += stats.bytesOutCounter; + this.msgOutCounter += stats.msgOutCounter; + this.waitingPublishers += stats.waitingPublishers; + double newAverageMsgSize = (this.averageMsgSize * (this.count - 1) + stats.averageMsgSize) / this.count; + this.averageMsgSize = newAverageMsgSize; + this.storageSize += stats.storageSize; + this.backlogSize += stats.backlogSize; + this.offloadedStorageSize += stats.offloadedStorageSize; + this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; + this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; + if (this.publishers.size() != stats.publishers.size()) { + for (int i = 0; i < stats.publishers.size(); i++) { + PublisherStatsImpl publisherStats = new PublisherStatsImpl(); + this.publishers.add(publisherStats.add(stats.publishers.get(i))); + } + } else { + for (int i = 0; i < stats.publishers.size(); i++) { + this.publishers.get(i).add(stats.publishers.get(i)); + } + } + if (this.subscriptions.size() != stats.subscriptions.size()) { + for (String subscription : stats.subscriptions.keySet()) { + SubscriptionStatsImpl subscriptionStats = new SubscriptionStatsImpl(); + this.subscriptions.put(subscription, subscriptionStats.add(stats.subscriptions.get(subscription))); + } + } else { + for (String subscription : stats.subscriptions.keySet()) { + if (this.subscriptions.get(subscription) != null) { + this.subscriptions.get(subscription).add(stats.subscriptions.get(subscription)); + } else { + SubscriptionStatsImpl subscriptionStats = new SubscriptionStatsImpl(); + this.subscriptions.put(subscription, subscriptionStats.add(stats.subscriptions.get(subscription))); + } + } + } + if (this.replication.size() != stats.replication.size()) { + for (String repl : stats.replication.keySet()) { + ReplicatorStatsImpl replStats = new ReplicatorStatsImpl(); + this.replication.put(repl, replStats.add(stats.replication.get(repl))); + } + } else { + for (String repl : stats.replication.keySet()) { + if (this.replication.get(repl) != null) { + this.replication.get(repl).add(stats.replication.get(repl)); + } else { + ReplicatorStatsImpl replStats = new ReplicatorStatsImpl(); + this.replication.put(repl, replStats.add(stats.replication.get(repl))); + } + } + } + return this; + } + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicies.java index 323ad0c1a9fbe..eb32addef6567 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicies.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicies.java @@ -25,6 +25,7 @@ import org.apache.pulsar.common.policies.NamespaceIsolationPolicy; import org.apache.pulsar.common.policies.data.BrokerAssignment; import org.apache.pulsar.common.policies.data.BrokerStatus; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; /** @@ -35,7 +36,7 @@ public class NamespaceIsolationPolicies { private Map policies = null; public NamespaceIsolationPolicies() { - policies = new HashMap(); + policies = new HashMap<>(); } public NamespaceIsolationPolicies(Map policiesMap) { @@ -65,7 +66,7 @@ public NamespaceIsolationPolicy getPolicyByName(String policyName) { * @return */ public NamespaceIsolationPolicy getPolicyByNamespace(NamespaceName namespace) { - for (NamespaceIsolationDataImpl nsPolicyData : policies.values()) { + for (NamespaceIsolationData nsPolicyData : policies.values()) { if (this.namespaceMatches(namespace, nsPolicyData)) { return new NamespaceIsolationPolicyImpl(nsPolicyData); } @@ -73,8 +74,8 @@ public NamespaceIsolationPolicy getPolicyByNamespace(NamespaceName namespace) { return null; } - private boolean namespaceMatches(NamespaceName namespace, NamespaceIsolationDataImpl nsPolicyData) { - for (String nsnameRegex : nsPolicyData.namespaces) { + private boolean namespaceMatches(NamespaceName namespace, NamespaceIsolationData nsPolicyData) { + for (String nsnameRegex : nsPolicyData.getNamespaces()) { if (namespace.toString().matches(nsnameRegex)) { return true; } @@ -88,9 +89,9 @@ private boolean namespaceMatches(NamespaceName namespace, NamespaceIsolationData * @param policyName * @param policyData */ - public void setPolicy(String policyName, NamespaceIsolationDataImpl policyData) { + public void setPolicy(String policyName, NamespaceIsolationData policyData) { policyData.validate(); - policies.put(policyName, policyData); + policies.put(policyName, (NamespaceIsolationDataImpl) policyData); } /** @@ -118,7 +119,7 @@ public Map getPolicies() { * @return */ public boolean isSharedBroker(String host) { - for (NamespaceIsolationDataImpl policyData : this.policies.values()) { + for (NamespaceIsolationData policyData : this.policies.values()) { NamespaceIsolationPolicyImpl policy = new NamespaceIsolationPolicyImpl(policyData); if (policy.isPrimaryBroker(host)) { // not free for sharing, this is some properties' primary broker diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java index 8c2af70f88d4d..b77dcb3306e89 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.policies.AutoFailoverPolicy; import org.apache.pulsar.common.policies.NamespaceIsolationPolicy; import org.apache.pulsar.common.policies.data.BrokerStatus; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; /** @@ -59,11 +60,11 @@ private List getMatchedBrokers(List brkRegexList, List availab return matchedBrokers; } - public NamespaceIsolationPolicyImpl(NamespaceIsolationDataImpl policyData) { - this.namespaces = policyData.namespaces; - this.primary = policyData.primary; - this.secondary = policyData.secondary; - this.autoFailoverPolicy = AutoFailoverPolicyFactory.create(policyData.auto_failover_policy); + public NamespaceIsolationPolicyImpl(NamespaceIsolationData policyData) { + this.namespaces = policyData.getNamespaces(); + this.primary = policyData.getPrimary(); + this.secondary = policyData.getSecondary(); + this.autoFailoverPolicy = AutoFailoverPolicyFactory.create(policyData.getAutoFailoverPolicy()); } @Override diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java index c37a0b18c2ea0..ffab615600068 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java @@ -27,17 +27,30 @@ import io.netty.util.concurrent.FastThreadLocal; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.ClassUtils; +import org.apache.pulsar.client.admin.internal.data.AuthPoliciesImpl; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionState; import org.apache.pulsar.common.functions.JsonIgnorePropertiesMixIn; +import org.apache.pulsar.common.policies.data.AuthPolicies; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; +import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuotaMixIn; +import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; +import org.apache.pulsar.common.policies.data.BookieInfo; +import org.apache.pulsar.common.policies.data.BookiesClusterInfo; +import org.apache.pulsar.common.policies.data.BrokerInfo; import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationData; +import org.apache.pulsar.common.policies.data.BrokerStatus; +import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FunctionInstanceStatsImpl; @@ -50,12 +63,42 @@ import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; +import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats; +import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats; +import org.apache.pulsar.common.policies.data.NonPersistentTopicStats; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.PublisherStats; +import org.apache.pulsar.common.policies.data.ReplicatorStats; import org.apache.pulsar.common.policies.data.ResourceQuota; import org.apache.pulsar.common.policies.data.ResourceQuotaMixIn; +import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.impl.AutoSubscriptionCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.BookieAffinityGroupDataImpl; +import org.apache.pulsar.common.policies.data.impl.BookieInfoImpl; +import org.apache.pulsar.common.policies.data.impl.BookiesClusterInfoImpl; +import org.apache.pulsar.common.policies.data.impl.BrokerInfoImpl; +import org.apache.pulsar.common.policies.data.impl.BrokerStatusImpl; +import org.apache.pulsar.common.policies.data.impl.BundlesDataImpl; +import org.apache.pulsar.common.policies.data.impl.DelayedDeliveryPoliciesImpl; +import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentSubscriptionStatsImpl; +import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PartitionedTopicStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.stats.MetricsMixIn; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; @@ -113,6 +156,7 @@ private static void setAnnotationsModule(ObjectMapper mapper) { SimpleAbstractTypeResolver resolver = new SimpleAbstractTypeResolver(); resolver.addMapping(AutoFailoverPolicyData.class, AutoFailoverPolicyDataImpl.class); resolver.addMapping(BrokerNamespaceIsolationData.class, BrokerNamespaceIsolationDataImpl.class); + resolver.addMapping(BacklogQuota.class, BacklogQuotaImpl.class); resolver.addMapping(ClusterData.class, ClusterDataImpl.class); resolver.addMapping(FailureDomain.class, FailureDomainImpl.class); resolver.addMapping(NamespaceIsolationData.class, NamespaceIsolationDataImpl.class); @@ -122,9 +166,30 @@ private static void setAnnotationsModule(ObjectMapper mapper) { resolver.addMapping(FunctionInstanceStats.class, FunctionInstanceStatsImpl.class); resolver.addMapping(FunctionInstanceStatsData.class, FunctionInstanceStatsDataImpl.class); resolver.addMapping(FunctionInstanceStatsDataBase.class, FunctionInstanceStatsDataBaseImpl.class); + resolver.addMapping(BundlesData.class, BundlesDataImpl.class); + resolver.addMapping(BookieAffinityGroupData.class, BookieAffinityGroupDataImpl.class); + resolver.addMapping(AuthPolicies.class, AuthPoliciesImpl.class); + resolver.addMapping(AutoTopicCreationOverride.class, AutoTopicCreationOverrideImpl.class); + resolver.addMapping(BookieInfo.class, BookieInfoImpl.class); + resolver.addMapping(BookiesClusterInfo.class, BookiesClusterInfoImpl.class); + resolver.addMapping(BrokerInfo.class, BrokerInfoImpl.class); + resolver.addMapping(BrokerStatus.class, BrokerStatusImpl.class); + resolver.addMapping(DelayedDeliveryPolicies.class, DelayedDeliveryPoliciesImpl.class); + resolver.addMapping(DispatchRate.class, DispatchRateImpl.class); + resolver.addMapping(TopicStats.class, TopicStatsImpl.class); + resolver.addMapping(ConsumerStats.class, ConsumerStatsImpl.class); + resolver.addMapping(NonPersistentPublisherStats.class, NonPersistentPublisherStatsImpl.class); + resolver.addMapping(NonPersistentReplicatorStats.class, NonPersistentReplicatorStatsImpl.class); + resolver.addMapping(NonPersistentSubscriptionStats.class, NonPersistentSubscriptionStatsImpl.class); + resolver.addMapping(NonPersistentTopicStats.class, NonPersistentTopicStatsImpl.class); + resolver.addMapping(PartitionedTopicStats.class, PartitionedTopicStatsImpl.class); + resolver.addMapping(PublisherStats.class, PublisherStatsImpl.class); + resolver.addMapping(ReplicatorStats.class, ReplicatorStatsImpl.class); + resolver.addMapping(SubscriptionStats.class, SubscriptionStatsImpl.class); + resolver.addMapping(AutoSubscriptionCreationOverride.class, AutoSubscriptionCreationOverrideImpl.class); // we use MixIn class to add jackson annotations - mapper.addMixIn(BacklogQuota.class, BacklogQuotaMixIn.class); + mapper.addMixIn(BacklogQuotaImpl.class, BacklogQuotaMixIn.class); mapper.addMixIn(ResourceQuota.class, ResourceQuotaMixIn.class); mapper.addMixIn(FunctionConfig.class, JsonIgnorePropertiesMixIn.class); mapper.addMixIn(FunctionState.class, JsonIgnorePropertiesMixIn.class); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataTest.java index 7842a38ec17dc..b6b615f6a5461 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoFailoverPolicyDataTest.java @@ -23,6 +23,7 @@ import static org.testng.Assert.fail; import java.util.HashMap; +import java.util.Map; import org.testng.annotations.Test; @@ -30,16 +31,23 @@ public class AutoFailoverPolicyDataTest { @Test public void testAutoFailoverPolicyData() { - AutoFailoverPolicyDataImpl policy0 = new AutoFailoverPolicyDataImpl(); - AutoFailoverPolicyDataImpl policy1 = new AutoFailoverPolicyDataImpl(); - policy0.policy_type = AutoFailoverPolicyType.min_available; - policy0.parameters = new HashMap<>(); - policy0.parameters.put("min_limit", "3"); - policy0.parameters.put("usage_threshold", "10"); - policy1.policy_type = AutoFailoverPolicyType.min_available; - policy1.parameters = new HashMap<>(); - policy1.parameters.put("min_limit", "3"); - policy1.parameters.put("usage_threshold", "10"); + Map p1parameters = new HashMap<>(); + p1parameters.put("min_limit", "3"); + p1parameters.put("usage_threshold", "10"); + + Map p2parameters = new HashMap<>(); + p2parameters.put("min_limit", "3"); + p2parameters.put("usage_threshold", "10"); + + AutoFailoverPolicyData policy0 = AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(p1parameters) + .build(); + AutoFailoverPolicyData policy1 = AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(p2parameters) + .build(); + try { policy0.validate(); policy1.validate(); @@ -47,7 +55,7 @@ public void testAutoFailoverPolicyData() { fail("Should not happen"); } assertEquals(policy1, policy0); - policy1.parameters.put("min_limit", "5"); + p1parameters.put("min_limit", "5"); assertNotEquals(policy1, policy0); assertNotEquals(new OldPolicies(), policy1); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java index 75bc110a6ae2b..5092d433d0db7 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.policies.data; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; import org.testng.annotations.Test; import static org.testng.Assert.assertFalse; @@ -27,37 +28,58 @@ public class AutoTopicCreationOverrideTest { @Test public void testValidOverrideNonPartitioned() { - AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null); - assertTrue(AutoTopicCreationOverride.isValidOverride(override)); + AutoTopicCreationOverride override = AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.NON_PARTITIONED.toString()) + .build(); + assertTrue(AutoTopicCreationOverrideImpl.isValidOverride(override)); } @Test public void testValidOverridePartitioned() { - AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 2); - assertTrue(AutoTopicCreationOverride.isValidOverride(override)); + AutoTopicCreationOverride override = AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .defaultNumPartitions(2) + .build(); + assertTrue(AutoTopicCreationOverrideImpl.isValidOverride(override)); } @Test public void testInvalidTopicType() { - AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, "aaa", null); - assertFalse(AutoTopicCreationOverride.isValidOverride(override)); + AutoTopicCreationOverride override = AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType("aaa") + .build(); + assertFalse(AutoTopicCreationOverrideImpl.isValidOverride(override)); } @Test public void testNumPartitionsTooLow() { - AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 0); - assertFalse(AutoTopicCreationOverride.isValidOverride(override)); + AutoTopicCreationOverride override = AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .defaultNumPartitions(0) + .build(); + assertFalse(AutoTopicCreationOverrideImpl.isValidOverride(override)); } @Test public void testNumPartitionsNotSet() { - AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), null); - assertFalse(AutoTopicCreationOverride.isValidOverride(override)); + AutoTopicCreationOverride override = AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .build(); + assertFalse(AutoTopicCreationOverrideImpl.isValidOverride(override)); } @Test public void testNumPartitionsOnNonPartitioned() { - AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), 2); - assertFalse(AutoTopicCreationOverride.isValidOverride(override)); + AutoTopicCreationOverride override = AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType(TopicType.NON_PARTITIONED.toString()) + .defaultNumPartitions(2) + .build(); + assertFalse(AutoTopicCreationOverrideImpl.isValidOverride(override)); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/BacklogQuotaTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/BacklogQuotaTest.java index 8f9f3aa9996cc..2a22dc24e5783 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/BacklogQuotaTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/BacklogQuotaTest.java @@ -25,16 +25,46 @@ public class BacklogQuotaTest { @Test public void testBacklogQuotaIdentity() { - Assert.assertNotEquals(new BacklogQuota(1, RetentionPolicy.producer_exception), - new BacklogQuota(2, RetentionPolicy.producer_exception)); - Assert.assertNotEquals(new BacklogQuota(1, RetentionPolicy.producer_exception), - new BacklogQuota(2, RetentionPolicy.consumer_backlog_eviction)); - Assert.assertNotEquals(new BacklogQuota(2, RetentionPolicy.producer_exception), - new BacklogQuota(2, RetentionPolicy.consumer_backlog_eviction)); - Assert.assertEquals(new BacklogQuota(10, RetentionPolicy.producer_exception), - new BacklogQuota(10, RetentionPolicy.producer_exception)); - BacklogQuota quota1 = new BacklogQuota(10, RetentionPolicy.producer_exception); - BacklogQuota quota2 = new BacklogQuota(10, RetentionPolicy.producer_exception); + Assert.assertNotEquals(BacklogQuota.builder() + .limitSize(1) + .retentionPolicy(RetentionPolicy.producer_exception) + .build(), + BacklogQuota.builder() + .limitSize(2) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); + Assert.assertNotEquals(BacklogQuota.builder() + .limitSize(1) + .retentionPolicy(RetentionPolicy.producer_exception) + .build(), + BacklogQuota.builder() + .limitSize(2) + .retentionPolicy(RetentionPolicy.consumer_backlog_eviction) + .build()); + Assert.assertNotEquals(BacklogQuota.builder() + .limitSize(2) + .retentionPolicy(RetentionPolicy.producer_exception) + .build(), + BacklogQuota.builder() + .limitSize(1) + .retentionPolicy(RetentionPolicy.consumer_backlog_eviction) + .build()); + Assert.assertEquals(BacklogQuota.builder() + .limitSize(10) + .retentionPolicy(RetentionPolicy.producer_exception) + .build(), + BacklogQuota.builder() + .limitSize(10) + .retentionPolicy(RetentionPolicy.producer_exception) + .build()); + BacklogQuota quota1 = BacklogQuota.builder() + .limitSize(10) + .retentionPolicy(RetentionPolicy.producer_exception) + .build(); + BacklogQuota quota2 = BacklogQuota.builder() + .limitSize(10) + .retentionPolicy(RetentionPolicy.producer_exception) + .build(); Assert.assertEquals(quota1.hashCode(), quota2.hashCode()); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataTest.java index 77e867e05e1e1..9689d3f06dce6 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataTest.java @@ -31,33 +31,82 @@ public void simple() { String s2 = "http://broker.messaging.c2.example.com:8080"; String s3 = "https://broker.messaging.c1.example.com:4443"; String s4 = "https://broker.messaging.c2.example.com:4443"; - ClusterDataImpl c = new ClusterDataImpl(s1); - c.setServiceUrl(null); - c.setServiceUrlTls(null); + ClusterData c = ClusterData.builder() + .serviceUrl(null) + .serviceUrlTls(null) + .build(); - assertEquals(new ClusterDataImpl(s1), new ClusterDataImpl(s1)); - assertEquals(new ClusterDataImpl(s1).getServiceUrl(), s1); + assertEquals(ClusterData.builder().serviceUrl(s1).build(), ClusterData.builder().serviceUrl(s1).build()); + assertEquals(ClusterData.builder().serviceUrl(s1).build().getServiceUrl(), s1); - assertNotEquals(new ClusterDataImpl(), new ClusterDataImpl(s1)); - assertNotEquals(new ClusterDataImpl(s2), new ClusterDataImpl(s1)); - assertNotEquals(s1, new ClusterDataImpl(s1)); + assertNotEquals(ClusterData.builder().build(), + ClusterData.builder().serviceUrl(s1).build()); + assertNotEquals(ClusterData.builder().serviceUrl(s2).build(), + ClusterData.builder().serviceUrl(s1).build()); + assertNotEquals(s1, ClusterData.builder().serviceUrl(s1).build()); - assertEquals(new ClusterDataImpl(s1).hashCode(), new ClusterDataImpl(s1).hashCode()); + assertEquals(ClusterData.builder().serviceUrl(s1).build().hashCode(), + ClusterData.builder().serviceUrl(s1).build().hashCode()); - assertNotEquals(new ClusterDataImpl(s2).hashCode(), new ClusterDataImpl(s1).hashCode()); + assertNotEquals(ClusterData.builder().serviceUrl(s2).build().hashCode(), + ClusterData.builder().serviceUrl(s1).build().hashCode()); - assertNotEquals(c.hashCode(), new ClusterDataImpl(s1).hashCode()); + assertNotEquals(c.hashCode(), ClusterData.builder().serviceUrl(s1).build().hashCode()); - assertEquals(new ClusterDataImpl(s1, s3), new ClusterDataImpl(s1, s3)); - assertEquals(new ClusterDataImpl(s1, s3).getServiceUrl(), s1); - assertEquals(new ClusterDataImpl(s1, s3).getServiceUrlTls(), s3); + assertEquals(ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build(), + ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build()); + assertEquals(ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build().getServiceUrl(), s1); + assertEquals(ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build().getServiceUrlTls(), s3); - assertNotEquals(new ClusterDataImpl(), new ClusterDataImpl(s1, s3)); - assertNotEquals(new ClusterDataImpl(s2, s4), new ClusterDataImpl(s1, s3)); + assertNotEquals(ClusterData.builder().build(), ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build()); + assertNotEquals(ClusterData.builder() + .serviceUrl(s2) + .serviceUrlTls(s4) + .build(), + ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build()); - assertEquals(new ClusterDataImpl(s1, s3).hashCode(), new ClusterDataImpl(s1, s3).hashCode()); - assertNotEquals(new ClusterDataImpl(s2, s4).hashCode(), new ClusterDataImpl(s1, s3).hashCode()); - assertNotEquals(new ClusterDataImpl(s1, s4).hashCode(), new ClusterDataImpl(s1, s3).hashCode()); + assertEquals(ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build().hashCode(), + ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build().hashCode()); + assertNotEquals(ClusterData.builder() + .serviceUrl(s2) + .serviceUrlTls(s4) + .build().hashCode(), + ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build().hashCode()); + assertNotEquals(ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s4) + .build().hashCode(), + ClusterData.builder() + .serviceUrl(s1) + .serviceUrlTls(s3) + .build().hashCode()); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ConsumerStatsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ConsumerStatsTest.java index 853597d21247d..b4b27d8a6f89d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ConsumerStatsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ConsumerStatsTest.java @@ -21,13 +21,14 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.testng.annotations.Test; public class ConsumerStatsTest { @Test public void testConsumerStats() { - ConsumerStats stats = new ConsumerStats(); + ConsumerStatsImpl stats = new ConsumerStatsImpl(); assertNull(stats.getAddress()); assertNull(stats.getClientVersion()); assertNull(stats.getConnectedSince()); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/LocalPolicesTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/LocalPolicesTest.java index 02b9b9ad0ddd4..d547923db3846 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/LocalPolicesTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/LocalPolicesTest.java @@ -35,8 +35,6 @@ public class LocalPolicesTest { @Test public void testLocalPolices() { - LocalPolicies localPolicy0 = new LocalPolicies(); - LocalPolicies localPolicy1 = new LocalPolicies(); List boundaries0 = new ArrayList<>(); List boundaries1 = new ArrayList<>(); for (int i = 0; i < 5; i++) { @@ -44,14 +42,22 @@ public void testLocalPolices() { boundaries0.add(String.valueOf(i)); boundaries1.add(String.valueOf(i)); } - localPolicy0.bundles.setBoundaries(boundaries0); - localPolicy0.bundles.setNumBundles(boundaries0.size() - 1); - localPolicy1.bundles.setBoundaries(boundaries1); - localPolicy1.bundles.setNumBundles(boundaries1.size() - 1); + LocalPolicies localPolicy0 = new LocalPolicies(BundlesData.builder() + .boundaries(boundaries0) + .numBundles(boundaries0.size() - 1) + .build(), null, null); + LocalPolicies localPolicy1 = new LocalPolicies(BundlesData.builder() + .boundaries(boundaries1) + .numBundles(boundaries1.size() - 1) + .build(), null, null); + assertNotEquals(localPolicy1, localPolicy0); assertNotEquals(new OldPolicies(), localPolicy0); - localPolicy1.bundles.setBoundaries(boundaries0); - localPolicy1.bundles.setNumBundles(boundaries0.size() - 1); + + localPolicy1 = new LocalPolicies(BundlesData.builder() + .boundaries(boundaries0) + .numBundles(boundaries0.size() - 1) + .build(), null, null); assertEquals(localPolicy1, localPolicy0); } @@ -104,7 +110,10 @@ public void testMakeLocalPoliciesImmutableSerializationCompatibility() throws IO // check with set other fields - BookieAffinityGroupData bookieAffinityGroupData = new BookieAffinityGroupData("aaa","bbb"); + BookieAffinityGroupData bookieAffinityGroupData = BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary("aaa") + .bookkeeperAffinityGroupSecondary("bbb") + .build(); String namespaceAntiAffinityGroup = "namespace1,namespace2"; mutableLocalPolicies.bookieAffinityGroup = bookieAffinityGroupData; @@ -136,7 +145,10 @@ public void testMakeLocalPoliciesImmutableStringSerializationCompatibility() thr // check with set other fields - BookieAffinityGroupData bookieAffinityGroupData = new BookieAffinityGroupData("aaa","bbb"); + BookieAffinityGroupData bookieAffinityGroupData = BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary("aaa") + .bookkeeperAffinityGroupSecondary("bbb") + .build(); String namespaceAntiAffinityGroup = "namespace1,namespace2"; mutableLocalPolicies.bookieAffinityGroup = bookieAffinityGroupData; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataTest.java index 7df8d2268f287..bc35c58c2a207 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataTest.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.Map; import org.testng.annotations.Test; @@ -31,24 +32,26 @@ public class NamespaceIsolationDataTest { @Test public void testNamespaceIsolationData() { - NamespaceIsolationDataImpl n0 = new NamespaceIsolationDataImpl(); - NamespaceIsolationDataImpl n1 = new NamespaceIsolationDataImpl(); + NamespaceIsolationData n0 = NamespaceIsolationData.builder() + .namespaces(new ArrayList<>()) + .primary(new ArrayList<>()) + .secondary(new ArrayList<>()) + .build(); assertNotEquals(new OldPolicies(), n0); - n0.namespaces = new ArrayList<>(); - n0.primary = new ArrayList<>(); - n0.secondary = new ArrayList<>(); for (int i = 0; i < 5; i++) { - n0.namespaces.add(String.format("ns%d", i)); - n0.primary.add(String.format("p%d", i)); - n0.secondary.add(String.format("s%d", i)); + n0.getNamespaces().add(String.format("ns%d", i)); + n0.getPrimary().add(String.format("p%d", i)); + n0.getSecondary().add(String.format("s%d", i)); } - assertNotEquals(new NamespaceIsolationDataImpl(), n0); + assertNotEquals(NamespaceIsolationData.builder().build(), n0); - n1.namespaces = n0.namespaces; - n1.primary = n0.primary; - n1.secondary = n0.secondary; + NamespaceIsolationData n1 = NamespaceIsolationData.builder() + .namespaces(n0.getNamespaces()) + .primary(n0.getPrimary()) + .secondary(n0.getSecondary()) + .build(); assertEquals(n1, n0); try { @@ -59,17 +62,45 @@ public void testNamespaceIsolationData() { // pass } - AutoFailoverPolicyDataImpl policy0 = new AutoFailoverPolicyDataImpl(); - AutoFailoverPolicyDataImpl policy1 = new AutoFailoverPolicyDataImpl(); - policy0.policy_type = AutoFailoverPolicyType.min_available; - policy0.parameters = new HashMap<>(); - policy0.parameters.put("min_limit", "3"); - policy0.parameters.put("usage_threshold", "10"); - policy1.policy_type = AutoFailoverPolicyType.min_available; - policy1.parameters = new HashMap<>(policy0.parameters); - - n0.auto_failover_policy = policy0; - n1.auto_failover_policy = policy1; + Map p1parameters = new HashMap<>(); + p1parameters.put("min_limit", "3"); + p1parameters.put("usage_threshold", "10"); + + Map p2parameters = new HashMap<>(); + p2parameters.put("min_limit", "3"); + p2parameters.put("usage_threshold", "10"); + + AutoFailoverPolicyData policy0 = AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(p1parameters) + .build(); + AutoFailoverPolicyData policy1 = AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(p1parameters) + .build(); + + n0 = NamespaceIsolationData.builder() + .namespaces(new ArrayList<>()) + .primary(new ArrayList<>()) + .secondary(new ArrayList<>()) + .autoFailoverPolicy(policy0) + .build(); + assertNotEquals(new OldPolicies(), n0); + + for (int i = 0; i < 5; i++) { + n0.getNamespaces().add(String.format("ns%d", i)); + n0.getPrimary().add(String.format("p%d", i)); + n0.getSecondary().add(String.format("s%d", i)); + } + + assertNotEquals(NamespaceIsolationData.builder().build(), n0); + + n1 = NamespaceIsolationData.builder() + .namespaces(n0.getNamespaces()) + .primary(n0.getPrimary()) + .secondary(n0.getSecondary()) + .autoFailoverPolicy(policy1) + .build(); try { n0.validate(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OldPolicies.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OldPolicies.java index 5d6fd9a4fbb0e..9ec11c8972ede 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OldPolicies.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OldPolicies.java @@ -37,7 +37,7 @@ public class OldPolicies { public Map latency_stats_sample_rate; public OldPolicies() { - auth_policies = new AuthPolicies(); + auth_policies = AuthPolicies.builder().build(); replication_clusters = Lists.newArrayList(); backlog_quota_map = Maps.newHashMap(); persistence = null; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PartitionedTopicStatsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PartitionedTopicStatsTest.java index 96fc74bb3c59b..8d9627bf0e21e 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PartitionedTopicStatsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PartitionedTopicStatsTest.java @@ -20,22 +20,26 @@ import static org.testng.Assert.assertEquals; +import org.apache.pulsar.common.policies.data.stats.PartitionedTopicStatsImpl; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; import org.testng.annotations.Test; public class PartitionedTopicStatsTest { @Test public void testPartitionedTopicStats() { - PartitionedTopicStats partitionedTopicStats = new PartitionedTopicStats(); + PartitionedTopicStatsImpl partitionedTopicStats = new PartitionedTopicStatsImpl(); partitionedTopicStats.msgRateIn = 1; partitionedTopicStats.msgThroughputIn = 1; partitionedTopicStats.msgRateOut = 1; partitionedTopicStats.msgThroughputOut = 1; partitionedTopicStats.averageMsgSize = 1; partitionedTopicStats.storageSize = 1; - partitionedTopicStats.publishers.add(new PublisherStats()); - partitionedTopicStats.subscriptions.put("test_ns", new SubscriptionStats()); - partitionedTopicStats.replication.put("test_ns", new ReplicatorStats()); + partitionedTopicStats.publishers.add(new PublisherStatsImpl()); + partitionedTopicStats.subscriptions.put("test_ns", new SubscriptionStatsImpl()); + partitionedTopicStats.replication.put("test_ns", new ReplicatorStatsImpl()); partitionedTopicStats.metadata.partitions = 1; partitionedTopicStats.partitions.put("test", partitionedTopicStats); partitionedTopicStats.reset(); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PersistentTopicStatsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PersistentTopicStatsTest.java index dcfbd2099c5cf..6b9e1b4e96edd 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PersistentTopicStatsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PersistentTopicStatsTest.java @@ -24,13 +24,17 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.PublisherStats; import org.apache.pulsar.common.policies.data.ReplicatorStats; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; +import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.testng.annotations.Test; public class PersistentTopicStatsTest { @Test public void testPersistentTopicStats() { - TopicStats topicStats = new TopicStats(); + TopicStatsImpl topicStats = new TopicStatsImpl(); topicStats.msgRateIn = 1; topicStats.msgThroughputIn = 1; topicStats.msgRateOut = 1; @@ -38,10 +42,10 @@ public void testPersistentTopicStats() { topicStats.averageMsgSize = 1; topicStats.storageSize = 1; topicStats.offloadedStorageSize = 1; - topicStats.publishers.add(new PublisherStats()); - topicStats.subscriptions.put("test_ns", new SubscriptionStats()); - topicStats.replication.put("test_ns", new ReplicatorStats()); - TopicStats target = new TopicStats(); + topicStats.publishers.add(new PublisherStatsImpl()); + topicStats.subscriptions.put("test_ns", new SubscriptionStatsImpl()); + topicStats.replication.put("test_ns", new ReplicatorStatsImpl()); + TopicStatsImpl target = new TopicStatsImpl(); target.add(topicStats); assertEquals(topicStats.msgRateIn, 1.0); assertEquals(topicStats.msgThroughputIn, 1.0); @@ -68,29 +72,29 @@ public void testPersistentTopicStats() { @Test public void testPersistentTopicStatsAggregation() { - TopicStats topicStats1 = new TopicStats(); + TopicStatsImpl topicStats1 = new TopicStatsImpl(); topicStats1.msgRateIn = 1; topicStats1.msgThroughputIn = 1; topicStats1.msgRateOut = 1; topicStats1.msgThroughputOut = 1; topicStats1.averageMsgSize = 1; topicStats1.storageSize = 1; - topicStats1.publishers.add(new PublisherStats()); - topicStats1.subscriptions.put("test_ns", new SubscriptionStats()); - topicStats1.replication.put("test_ns", new ReplicatorStats()); + topicStats1.publishers.add(new PublisherStatsImpl()); + topicStats1.subscriptions.put("test_ns", new SubscriptionStatsImpl()); + topicStats1.replication.put("test_ns", new ReplicatorStatsImpl()); - TopicStats topicStats2 = new TopicStats(); + TopicStatsImpl topicStats2 = new TopicStatsImpl(); topicStats2.msgRateIn = 1; topicStats2.msgThroughputIn = 2; topicStats2.msgRateOut = 3; topicStats2.msgThroughputOut = 4; topicStats2.averageMsgSize = 5; topicStats2.storageSize = 6; - topicStats2.publishers.add(new PublisherStats()); - topicStats2.subscriptions.put("test_ns", new SubscriptionStats()); - topicStats2.replication.put("test_ns", new ReplicatorStats()); + topicStats2.publishers.add(new PublisherStatsImpl()); + topicStats2.subscriptions.put("test_ns", new SubscriptionStatsImpl()); + topicStats2.replication.put("test_ns", new ReplicatorStatsImpl()); - TopicStats target = new TopicStats(); + TopicStatsImpl target = new TopicStatsImpl(); target.add(topicStats1); target.add(topicStats2); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PoliciesDataTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PoliciesDataTest.java index d42a71e9edec0..ebc1d9d4dcc57 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PoliciesDataTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PoliciesDataTest.java @@ -45,29 +45,34 @@ public void policies() { assertEquals(policies, new Policies()); - policies.auth_policies.namespace_auth.put("my-role", EnumSet.of(AuthAction.consume)); + policies.auth_policies.getNamespaceAuthentication().put("my-role", EnumSet.of(AuthAction.consume)); assertNotEquals(new Policies(), policies); assertNotEquals(new Object(), policies); - policies.auth_policies.namespace_auth.clear(); + policies.auth_policies.getNamespaceAuthentication().clear(); Map> permissions = Maps.newTreeMap(); permissions.put("my-role", EnumSet.of(AuthAction.consume)); - policies.auth_policies.destination_auth.put("persistent://my-dest", permissions); + policies.auth_policies.getTopicAuthentication().put("persistent://my-dest", permissions); assertNotEquals(new Policies(), policies); } @Test public void propertyAdmin() { - TenantInfoImpl pa1 = new TenantInfoImpl(); - pa1.setAdminRoles(Sets.newHashSet("role1", "role2")); - pa1.setAllowedClusters(Sets.newHashSet("use", "usw")); - - assertEquals(pa1, new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("use", "usw"))); + TenantInfo pa1 = TenantInfo.builder() + .adminRoles(Sets.newHashSet("role1", "role2")) + .allowedClusters(Sets.newHashSet("use", "usw")) + .build(); + + assertEquals(pa1, TenantInfo.builder() + .adminRoles(Sets.newHashSet("role1", "role2")) + .allowedClusters(Sets.newHashSet("use", "usw")) + .build()); assertNotEquals(new Object(), pa1); - assertNotEquals(new TenantInfoImpl(), pa1); - assertNotEquals(new TenantInfoImpl(Sets.newHashSet("role1", "role3"), Sets.newHashSet("usc")), pa1); + assertNotEquals(TenantInfo.builder().build(), pa1); + assertNotEquals(TenantInfo.builder().adminRoles(Sets.newHashSet("role1", "role3")) + .allowedClusters(Sets.newHashSet("usc")).build(), pa1); assertEquals(pa1.getAdminRoles(), Lists.newArrayList("role1", "role2")); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java index 2bc4dd75ca4fd..6bc8771127bb9 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java @@ -21,13 +21,14 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; +import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; import org.testng.annotations.Test; public class PublisherStatsTest { @Test public void testPublisherStats() { - PublisherStats stats = new PublisherStats(); + PublisherStatsImpl stats = new PublisherStatsImpl(); assertNull(stats.getAddress()); assertNull(stats.getClientVersion()); assertNull(stats.getConnectedSince()); @@ -77,17 +78,17 @@ public void testPublisherStats() { @Test public void testPublisherStatsAggregation() { - PublisherStats stats1 = new PublisherStats(); + PublisherStatsImpl stats1 = new PublisherStatsImpl(); stats1.msgRateIn = 1; stats1.msgThroughputIn = 1; stats1.averageMsgSize = 1; - PublisherStats stats2 = new PublisherStats(); + PublisherStatsImpl stats2 = new PublisherStatsImpl(); stats2.msgRateIn = 1; stats2.msgThroughputIn = 2; stats2.averageMsgSize = 3; - PublisherStats target = new PublisherStats(); + PublisherStatsImpl target = new PublisherStatsImpl(); target.add(stats1); target.add(stats2); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ReplicatorStatsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ReplicatorStatsTest.java index 0467053c829a2..9e17e03de271f 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ReplicatorStatsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ReplicatorStatsTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.policies.data; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.testng.Assert; import org.testng.annotations.Test; @@ -25,7 +26,7 @@ public class ReplicatorStatsTest { @Test public void testReplicatorStatsNull() { - ReplicatorStats rs = new ReplicatorStats(); + ReplicatorStatsImpl rs = new ReplicatorStatsImpl(); try { rs.add(null); Assert.fail("Must fail."); @@ -36,7 +37,7 @@ public void testReplicatorStatsNull() { @Test public void testReplicatorStatsAdd() { - ReplicatorStats replicatorStats = new ReplicatorStats(); + ReplicatorStatsImpl replicatorStats = new ReplicatorStatsImpl(); replicatorStats.msgRateIn = 5; replicatorStats.msgThroughputIn = 10; replicatorStats.msgRateOut = 5; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactoryTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactoryTest.java index b08dbc3d01177..75e8889e435ff 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactoryTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/AutoFailoverPolicyFactoryTest.java @@ -19,8 +19,9 @@ package org.apache.pulsar.common.policies.impl; import java.util.HashMap; +import java.util.Map; -import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.testng.Assert; import org.testng.annotations.Test; @@ -30,17 +31,20 @@ public class AutoFailoverPolicyFactoryTest { @Test public void testAutoFailoverPolicyFactory() { try { - AutoFailoverPolicyFactory.create(new AutoFailoverPolicyDataImpl()); + AutoFailoverPolicyFactory.create(AutoFailoverPolicyData.builder().build()); Assert.fail(""); } catch (IllegalArgumentException e) { // Pass } try { - AutoFailoverPolicyDataImpl afopd = new AutoFailoverPolicyDataImpl(); - afopd.policy_type = AutoFailoverPolicyType.min_available; - afopd.parameters = new HashMap<>(); - afopd.parameters.put("min_limit", "3"); - afopd.parameters.put("usage_threshold", "10"); + Map parameters = new HashMap<>(); + parameters.put("min_limit", "3"); + parameters.put("usage_threshold", "10"); + + AutoFailoverPolicyData afopd = AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build(); AutoFailoverPolicyFactory.create(afopd); // Pass } catch (IllegalArgumentException e) { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java index d9f08796f5edb..fb58f2e6efacf 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java @@ -24,29 +24,28 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; - -import java.util.ArrayList; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.core.type.TypeReference; - import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.NamespaceIsolationPolicy; -import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BrokerStatus; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.annotations.Test; public class NamespaceIsolationPoliciesTest { - private final String defaultJson = "{\"policy1\":{\"namespaces\":[\"pulsar/use/test.*\"],\"primary\":[\"prod1-broker[1-3].messaging.use.example.com\"],\"secondary\":[\"prod1-broker.*.use.example.com\"],\"auto_failover_policy\":{\"policy_type\":\"min_available\",\"parameters\":{\"min_limit\":\"3\",\"usage_threshold\":\"100\"}}}}"; + private final String defaultJson = + "{\"policy1\":{\"namespaces\":[\"pulsar/use/test.*\"],\"primary\":[\"prod1-broker[1-3].messaging.use.example.com\"],\"secondary\":[\"prod1-broker.*.use.example.com\"],\"auto_failover_policy\":{\"parameters\":{\"min_limit\":\"3\",\"usage_threshold\":\"100\"},\"policy_type\":\"min_available\"}}}"; @Test public void testJsonSerialization() throws Exception { @@ -67,18 +66,19 @@ public void testJsonSerialization() throws Exception { byte[] outJson = jsonMapperForWriter.writeValueAsBytes(policies.getPolicies()); assertEquals(new String(outJson), this.defaultJson); - NamespaceIsolationDataImpl nsPolicyData = new NamespaceIsolationDataImpl(); - nsPolicyData.namespaces = new ArrayList<>(); - nsPolicyData.namespaces.add("other/use/other.*"); - nsPolicyData.primary = new ArrayList<>(); - nsPolicyData.primary.add("prod1-broker[4-6].messaging.use.example.com"); - nsPolicyData.secondary = new ArrayList<>(); - nsPolicyData.secondary.add("prod1-broker.*.messaging.use.example.com"); - nsPolicyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - nsPolicyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - nsPolicyData.auto_failover_policy.parameters = new HashMap<>(); - nsPolicyData.auto_failover_policy.parameters.put("min_limit", "1"); - nsPolicyData.auto_failover_policy.parameters.put("usage_threshold", "100"); + Map parameters = new HashMap<>(); + parameters.put("min_limit", "1"); + parameters.put("usage_threshold", "100"); + + NamespaceIsolationData nsPolicyData = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("pulsar/use/other.*")) + .primary(Collections.singletonList("prod1-broker[4-6].messaging.use.example.com")) + .secondary(Collections.singletonList("prod1-broker.*.messaging.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build() + ).build(); policies.setPolicy("otherPolicy", nsPolicyData); byte[] morePolicyJson = jsonMapperForWriter.writeValueAsBytes(policies.getPolicies()); ObjectMapper jsonParser = ObjectMapperFactory.create(); @@ -158,9 +158,21 @@ public void testBrokerAssignment() throws Exception { NamespaceIsolationPolicies policies = this.getDefaultTestPolicies(); NamespaceName ns = NamespaceName.get("pulsar/use/testns-1"); SortedSet primaryCandidates = new TreeSet<>(); - BrokerStatus primary = new BrokerStatus("prod1-broker1.messaging.use.example.com", true, 0); - BrokerStatus secondary = new BrokerStatus("prod1-broker4.use.example.com", true, 0); - BrokerStatus shared = new BrokerStatus("use.example.com", true, 0); + BrokerStatus primary = BrokerStatus.builder() + .brokerAddress("prod1-broker1.messaging.use.example.com") + .active(true) + .loadFactor(0) + .build(); + BrokerStatus secondary = BrokerStatus.builder() + .brokerAddress("prod1-broker4.messaging.use.example.com") + .active(true) + .loadFactor(0) + .build(); + BrokerStatus shared = BrokerStatus.builder() + .brokerAddress("use.example.com") + .active(true) + .loadFactor(0) + .build(); SortedSet secondaryCandidates = new TreeSet<>(); SortedSet sharedCandidates = new TreeSet<>(); policies.assignBroker(ns, primary, primaryCandidates, secondaryCandidates, sharedCandidates); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java index 5177b26d97678..2219cd0dbc46f 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java @@ -26,17 +26,21 @@ import java.net.URL; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyDataImpl; import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; import org.apache.pulsar.common.policies.data.BrokerStatus; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.OldPolicies; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -48,27 +52,30 @@ public class NamespaceIsolationPolicyImplTest { private NamespaceIsolationPolicyImpl getDefaultPolicy() throws Exception { ObjectMapper jsonMapper = ObjectMapperFactory.create(); return new NamespaceIsolationPolicyImpl( - jsonMapper.readValue(this.defaultPolicyJson.getBytes(), NamespaceIsolationDataImpl.class)); + jsonMapper.readValue(this.defaultPolicyJson.getBytes(), NamespaceIsolationData.class)); } @Test public void testConstructor() throws Exception { NamespaceIsolationPolicyImpl defaultPolicy = this.getDefaultPolicy(); - NamespaceIsolationDataImpl policyData = new NamespaceIsolationDataImpl(); - policyData.namespaces = new ArrayList<>(); - policyData.namespaces.add("pulsar/use/test.*"); - policyData.primary = new ArrayList<>(); - policyData.primary.add("prod1-broker[1-3].messaging.use.example.com"); - policyData.secondary = new ArrayList<>(); - policyData.secondary.add("prod1-broker.*.use.example.com"); - policyData.auto_failover_policy = new AutoFailoverPolicyDataImpl(); - policyData.auto_failover_policy.policy_type = AutoFailoverPolicyType.min_available; - policyData.auto_failover_policy.parameters = new HashMap<>(); - policyData.auto_failover_policy.parameters.put("min_limit", "3"); - policyData.auto_failover_policy.parameters.put("usage_threshold", "90"); + + Map parameters = new HashMap<>(); + parameters.put("min_limit", "3"); + parameters.put("usage_threshold", "90"); + + NamespaceIsolationData policyData = NamespaceIsolationData.builder() + .namespaces(Collections.singletonList("pulsar/use/test.*")) + .primary(Collections.singletonList("prod1-broker[1-3].messaging.use.example.com")) + .secondary(Collections.singletonList("prod1-broker.*.use.example.com")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters) + .build() + ).build(); + NamespaceIsolationPolicyImpl newPolicy = new NamespaceIsolationPolicyImpl(policyData); assertEquals(newPolicy, defaultPolicy); - policyData.auto_failover_policy.parameters.put("usage_threshold", "80"); + parameters.put("usage_threshold", "80"); newPolicy = new NamespaceIsolationPolicyImpl(policyData); assertNotEquals(newPolicy, defaultPolicy); assertNotEquals(new OldPolicies(), newPolicy); @@ -140,27 +147,49 @@ public void testFindBrokers() throws Exception { @Test public void testShouldFailover() throws Exception { NamespaceIsolationPolicyImpl defaultPolicy = this.getDefaultPolicy(); - SortedSet brokerStatus = new TreeSet<>(); + List brokerStatus = new ArrayList<>(); for (int i = 0; i < 10; i++) { - BrokerStatus status = new BrokerStatus(String.format("broker-%d", i), true, i * 10); + BrokerStatus status = BrokerStatus.builder() + .brokerAddress(String.format("broker-%s", i)) + .active(true) + .loadFactor(i * 10) + .build(); brokerStatus.add(status); } - assertFalse(defaultPolicy.shouldFailover(brokerStatus)); - List objList = new ArrayList<>(brokerStatus); + assertFalse(defaultPolicy.shouldFailover(new TreeSet<>(brokerStatus))); for (int i = 0; i < 8; i++) { - objList.get(i).setActive(false); + brokerStatus.set(i, BrokerStatus.builder() + .brokerAddress(brokerStatus.get(i).getBrokerAddress()) + .active(false) + .loadFactor(brokerStatus.get(i).getLoadFactor()) + .build()); } - assertTrue(defaultPolicy.shouldFailover(brokerStatus)); - objList.get(7).setActive(true); - assertTrue(defaultPolicy.shouldFailover(brokerStatus)); - objList.get(9).setLoadFactor(80); - assertFalse(defaultPolicy.shouldFailover(brokerStatus)); - - brokerStatus = new TreeSet<>(); + assertTrue(defaultPolicy.shouldFailover(new TreeSet<>(brokerStatus))); + brokerStatus.set(7, BrokerStatus.builder() + .brokerAddress(brokerStatus.get(7).getBrokerAddress()) + .active(true) + .loadFactor(brokerStatus.get(7).getLoadFactor()) + .build()); + assertTrue(defaultPolicy.shouldFailover(new TreeSet<>(brokerStatus))); + brokerStatus.set(9, BrokerStatus.builder() + .brokerAddress(brokerStatus.get(9).getBrokerAddress()) + .active(brokerStatus.get(9).isActive()) + .loadFactor(80) + .build()); + assertFalse(defaultPolicy.shouldFailover(new TreeSet<>(brokerStatus))); + + brokerStatus = new ArrayList<>(); for (int i = 0; i < 5; i++) { - BrokerStatus status = new BrokerStatus(String.format("broker-%d", 2 * i), true, i * 20); + BrokerStatus status = BrokerStatus.builder().brokerAddress(String.format("broker-%d", 2 * i)) + .active(true) + .loadFactor(i * 20) + .build(); brokerStatus.add(status); - status = new BrokerStatus(String.format("broker-%d", 2 * i + 1), true, i * 20); + status = BrokerStatus.builder() + .brokerAddress(String.format("broker-%d", 2 * i + 1)) + .active(true) + .loadFactor(i * 20) + .build(); brokerStatus.add(status); } assertEquals(brokerStatus.size(), 10); @@ -172,8 +201,11 @@ public void testGetAvailablePrimaryBrokers() throws Exception { SortedSet brokerStatus = new TreeSet<>(); SortedSet expectedAvailablePrimaries = new TreeSet<>(); for (int i = 0; i < 10; i++) { - BrokerStatus status = new BrokerStatus(String.format("prod1-broker%d.messaging.use.example.com", i), - i % 2 == 0, i * 10); + BrokerStatus status = BrokerStatus.builder() + .brokerAddress(String.format("prod1-broker%d.messaging.use.example.com", i)) + .active(i % 2 == 0) + .loadFactor(i * 10) + .build(); brokerStatus.add(status); if (i % 2 == 0) { expectedAvailablePrimaries.add(status); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/ObjectMapperFactoryTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/ObjectMapperFactoryTest.java index e42e6987c942c..dc8aa8b43fcf3 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/ObjectMapperFactoryTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/ObjectMapperFactoryTest.java @@ -19,8 +19,10 @@ package org.apache.pulsar.common.util; import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.ToString; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.ResourceQuota; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; import org.apache.pulsar.common.stats.Metrics; import org.testng.Assert; import org.testng.annotations.Test; @@ -41,8 +43,11 @@ public void testBacklogQuotaMixIn() { try { String expectJson = "{\"limitSize\":10,\"limitTime\":0,\"policy\":\"producer_request_hold\"}"; - BacklogQuota backlogQuota = new BacklogQuota(10, 0, - BacklogQuota.RetentionPolicy.producer_request_hold); + BacklogQuota backlogQuota = BacklogQuota.builder() + .limitSize(10) + .limitTime(0) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_request_hold) + .build(); String writeJson = objectMapper.writeValueAsString(backlogQuota); Assert.assertEquals(expectJson, writeJson); } catch (Exception ex) { diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java index be8abf2e54364..7f3eb6cfa066c 100644 --- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java +++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java @@ -36,6 +36,7 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.discovery.service.server.ServiceConfig; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; @@ -138,7 +139,7 @@ protected static void checkAuthorization(DiscoveryService service, TopicName top if (!service.getAuthorizationService().canLookup(topicName, role, authenticationData)) { LOG.warn("[{}] Role {} is not allowed to lookup topic", topicName, role); // check namespace authorization - TenantInfoImpl tenantInfo; + TenantInfo tenantInfo; try { tenantInfo = service.getPulsarResources().getTenantResources() .get(path(POLICIES, topicName.getTenant())) diff --git a/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java b/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java index 50fd63db0cdc2..dc9975f7cc9de 100644 --- a/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java +++ b/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java @@ -181,10 +181,10 @@ public void testRiderectUrlWithServerStarted() throws Exception { * must get unknown host exception with above brokers in a sequential manner. **/ - assertEquals(brokers, validateRequest(brokers, HttpMethod.PUT, requestUrl, new BundlesData(1)), + assertEquals(brokers, validateRequest(brokers, HttpMethod.PUT, requestUrl, BundlesData.builder().numBundles(1).build()), "redirection failed"); assertEquals(brokers, validateRequest(brokers, HttpMethod.GET, requestUrl, null), "redirection failed"); - assertEquals(brokers, validateRequest(brokers, HttpMethod.POST, requestUrl, new BundlesData(1)), + assertEquals(brokers, validateRequest(brokers, HttpMethod.POST, requestUrl, BundlesData.builder().numBundles(1).build()), "redirection failed"); server.stop(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java index 5f4665a296264..6497e1547896e 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java @@ -384,10 +384,10 @@ private Supplier getDeleteSubscriptionSupplier( SubscriptionStats sub = null; try { TopicStats stats = pulsarAdmin.topics().getStats(topic); - sub = stats.subscriptions.get(subscriptionName); + sub = stats.getSubscriptions().get(subscriptionName); if (sub != null) { - existingConsumers = sub.consumers.stream() - .map(consumerStats -> consumerStats.metadata) + existingConsumers = sub.getConsumers().stream() + .map(consumerStats -> consumerStats.getMetadata()) .collect(Collectors.toList()); } } catch (PulsarAdminException e1) { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/MembershipManager.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/MembershipManager.java index bb5a2d060f035..74b300a541673 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/MembershipManager.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/MembershipManager.java @@ -78,9 +78,9 @@ public List getCurrentMembership() { throw new RuntimeException(e); } - for (ConsumerStats consumerStats : topicStats.subscriptions - .get(COORDINATION_TOPIC_SUBSCRIPTION).consumers) { - WorkerInfo workerInfo = WorkerInfo.parseFrom(consumerStats.metadata.get(WORKER_IDENTIFIER)); + for (ConsumerStats consumerStats : topicStats.getSubscriptions() + .get(COORDINATION_TOPIC_SUBSCRIPTION).getConsumers()) { + WorkerInfo workerInfo = WorkerInfo.parseFrom(consumerStats.getMetadata().get(WORKER_IDENTIFIER)); workerIds.add(workerInfo); } return workerIds; @@ -96,12 +96,12 @@ public WorkerInfo getLeader() { throw new RuntimeException(e); } - String activeConsumerName = topicStats.subscriptions.get(COORDINATION_TOPIC_SUBSCRIPTION).activeConsumerName; + String activeConsumerName = topicStats.getSubscriptions().get(COORDINATION_TOPIC_SUBSCRIPTION).getActiveConsumerName(); WorkerInfo leader = null; - for (ConsumerStats consumerStats : topicStats.subscriptions - .get(COORDINATION_TOPIC_SUBSCRIPTION).consumers) { - if (consumerStats.consumerName.equals(activeConsumerName)) { - leader = WorkerInfo.parseFrom(consumerStats.metadata.get(WORKER_IDENTIFIER)); + for (ConsumerStats consumerStats : topicStats.getSubscriptions() + .get(COORDINATION_TOPIC_SUBSCRIPTION).getConsumers()) { + if (consumerStats.getConsumerName().equals(activeConsumerName)) { + leader = WorkerInfo.parseFrom(consumerStats.getMetadata().get(WORKER_IDENTIFIER)); } } if (leader == null) { 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 a4313b6522219..85c1152b0d942 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 @@ -55,6 +55,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.conf.InternalConfigurationData; import org.apache.pulsar.common.naming.NamedEntity; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; @@ -318,11 +319,10 @@ public void initInBroker(ServiceConfiguration brokerConfig, // create cluster for function worker service try { NamedEntity.checkName(cluster); - ClusterDataImpl clusterData = new ClusterDataImpl( - workerConfig.getPulsarWebServiceUrl(), - null /* serviceUrlTls */, - workerConfig.getPulsarServiceUrl(), - null /* brokerServiceUrlTls */); + ClusterDataImpl clusterData = ClusterDataImpl.builder() + .serviceUrl(workerConfig.getPulsarWebServiceUrl()) + .brokerServiceUrl(workerConfig.getPulsarServiceUrl()) + .build(); pulsarResources.getClusterResources().create( PolicyPath.path("clusters", cluster), clusterData); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java index 3ea7fd3f8ffd6..ba8a13a3e3426 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java @@ -28,7 +28,7 @@ import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.policies.data.FunctionInstanceStatsImpl; import org.apache.pulsar.common.policies.data.FunctionStatsImpl; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.instance.InstanceConfig; import org.apache.pulsar.functions.instance.InstanceUtils; @@ -114,7 +114,7 @@ public String process(String input, Context context) { private PulsarAdmin mockedPulsarAdmin; private Tenants mockedTenants; private Namespaces mockedNamespaces; - private TenantInfoImpl mockedTenantInfo; + private TenantInfo mockedTenantInfo; private List namespaceList = new LinkedList<>(); private FunctionMetaDataManager mockedManager; private FunctionRuntimeManager mockedFunctionRunTimeManager; @@ -130,7 +130,7 @@ public String process(String input, Context context) { public void setup() throws Exception { this.mockedManager = mock(FunctionMetaDataManager.class); this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedTenantInfo = mock(TenantInfoImpl.class); + this.mockedTenantInfo = mock(TenantInfo.class); this.mockedRuntimeFactory = mock(RuntimeFactory.class); this.mockedInputStream = mock(InputStream.class); this.mockedNamespace = mock(Namespace.class); @@ -247,7 +247,7 @@ public void testMetricsEmpty() { @Test public void testIsAuthorizedRole() throws PulsarAdminException, InterruptedException, ExecutionException { - TenantInfoImpl tenantInfo = new TenantInfoImpl(); + TenantInfo tenantInfo = TenantInfo.builder().build(); AuthenticationDataSource authenticationDataSource = mock(AuthenticationDataSource.class); FunctionsImpl functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); AuthorizationService authorizationService = mock(AuthorizationService.class); @@ -283,7 +283,7 @@ public void testIsAuthorizedRole() throws PulsarAdminException, InterruptedExcep functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); doReturn(false).when(functionImpl).allowFunctionOps(any(), any(), any()); tenants = mock(Tenants.class); - tenantInfo.setAdminRoles(Collections.singleton("test-user")); + tenantInfo = TenantInfo.builder().adminRoles(Collections.singleton("test-user")).build(); when(tenants.getTenantInfo(any())).thenReturn(tenantInfo); admin = mock(PulsarAdmin.class); @@ -297,7 +297,7 @@ public void testIsAuthorizedRole() throws PulsarAdminException, InterruptedExcep functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); doReturn(true).when(functionImpl).allowFunctionOps(any(), any(), any()); tenants = mock(Tenants.class); - tenantInfo.setAdminRoles(Collections.emptySet()); + tenantInfo = TenantInfo.builder().build(); when(tenants.getTenantInfo(any())).thenReturn(tenantInfo); admin = mock(PulsarAdmin.class); @@ -310,7 +310,7 @@ public void testIsAuthorizedRole() throws PulsarAdminException, InterruptedExcep functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); doReturn(true).when(functionImpl).allowFunctionOps(any(), any(), any()); tenants = mock(Tenants.class); - when(tenants.getTenantInfo(any())).thenReturn(new TenantInfoImpl()); + when(tenants.getTenantInfo(any())).thenReturn(TenantInfo.builder().build()); admin = mock(PulsarAdmin.class); when(admin.tenants()).thenReturn(tenants); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java index b21c76a606163..545912dcdc030 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -157,7 +158,7 @@ public void testAuthenticatedProxyAsNonAdmin() throws Exception { // expected } - brokerAdmin.clusters().createCluster(configClusterName, new ClusterDataImpl(brokerUrl.toString())); + brokerAdmin.clusters().createCluster(configClusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); brokerAdmin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("user1"), diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java index 40f74641fbbde..e63d3aeb4cb96 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -167,8 +168,12 @@ public void testTlsSyncProducerAndConsumer() throws Exception { @Cleanup PulsarClient proxyClient = createPulsarClient(authTls, proxyServiceUrl); - admin.clusters().createCluster(configClusterName, new ClusterDataImpl(brokerUrl.toString(), brokerUrlTls.toString(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls())); + admin.clusters().createCluster(configClusterName, ClusterData.builder() + .serviceUrl(brokerUrl.toString()) + .serviceUrlTls(brokerUrlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java index 4691a37f02cc0..44403fbb39b16 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java @@ -105,15 +105,15 @@ public void testSimpleProduceAndConsume() throws PulsarClientException, PulsarAd Assert.assertEquals(received, messages); TopicStats topicStats = admin.topics().getStats(topicName); - Assert.assertEquals(topicStats.subscriptions.size(), 1); - SubscriptionStats subscriptionStats = topicStats.subscriptions.get(subName); - Assert.assertEquals(subscriptionStats.consumers.size(), 1); - Assert.assertEquals(subscriptionStats.consumers.get(0).getAddress(), + 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(), ((ConsumerImpl) consumer).getClientCnx().ctx().channel().localAddress().toString().replaceFirst("/", "")); topicStats = admin.topics().getStats(topicName); - Assert.assertEquals(topicStats.publishers.size(), 1); - Assert.assertEquals(topicStats.publishers.get(0).getAddress(), + Assert.assertEquals(topicStats.getPublishers().size(), 1); + Assert.assertEquals(topicStats.getPublishers().get(0).getAddress(), ((ProducerImpl) producer).getClientCnx().ctx().channel().localAddress().toString().replaceFirst("/", "")); } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java index eaba9c088d0e8..5d05867d4fffd 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.mockito.Mockito; @@ -168,7 +169,7 @@ public void testProxyAuthorization() throws Exception { String namespaceName = "my-property/proxy-authorization-neg/my-ns"; - admin.clusters().createCluster("proxy-authorization-neg", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization-neg", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization-neg"))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index 4a62b99c51fe1..14c72881b2994 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -45,6 +45,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.mockito.Mockito; @@ -240,7 +241,7 @@ public void testProxyAuthorization() throws Exception { String namespaceName = "my-property/proxy-authorization/my-ns"; - admin.clusters().createCluster("proxy-authorization", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization"))); @@ -294,7 +295,7 @@ public void testTlsHostVerificationProxyToClient(boolean hostnameVerificationEna String namespaceName = "my-property/proxy-authorization/my-ns"; - admin.clusters().createCluster("proxy-authorization", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization"))); @@ -346,7 +347,7 @@ public void testTlsHostVerificationProxyToBroker(boolean hostnameVerificationEna String namespaceName = "my-property/proxy-authorization/my-ns"; - admin.clusters().createCluster("proxy-authorization", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization"))); @@ -383,7 +384,7 @@ public void tlsCiphersAndProtocols(Set tlsCiphers, Set tlsProtoc String namespaceName = "my-property/proxy-authorization/my-ns"; createAdminClient(); - admin.clusters().createCluster("proxy-authorization", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization"))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index 788e914476679..693e4ca5db9d6 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -142,7 +143,7 @@ public void testProxyAuthorization() throws Exception { String namespaceName = "my-property/proxy-authorization/my-ns"; - admin.clusters().createCluster("proxy-authorization", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization"))); @@ -223,7 +224,7 @@ public void testUpdatePartitionNumAndReconnect() throws Exception { String topicName = "persistent://my-property/my-ns/my-topic1"; String subscriptionName = "my-subscriber-name"; - admin.clusters().createCluster(clusterName, new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster(clusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet(clusterName))); @@ -315,7 +316,7 @@ public void testProxyAuthorizationWithPrefixSubscriptionAuthMode() throws Except String namespaceName = "my-property/proxy-authorization/my-ns"; - admin.clusters().createCluster("proxy-authorization", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization"))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java index 774a5dbfa7e48..f20401c33aebf 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.mockito.Mockito; @@ -161,7 +162,7 @@ public void testDiscoveryService() throws Exception { @Cleanup PulsarClient proxyClient = createPulsarClient(authTls, proxyService.getServiceUrlTls()); - admin.clusters().createCluster("without-service-discovery", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("without-service-discovery", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("without-service-discovery"))); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java index def1bf63dc5c9..7dc927a8a5cc2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -134,7 +135,7 @@ PulsarAdmin getAdminClient(String user) throws Exception { @Test public void testAuthenticatedProxyAsAdmin() throws Exception { try (PulsarAdmin adminAdmin = getAdminClient("admin")) { - adminAdmin.clusters().createCluster(configClusterName, new ClusterDataImpl(brokerUrl.toString())); + adminAdmin.clusters().createCluster(configClusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); adminAdmin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("randoUser"), ImmutableSet.of(configClusterName))); @@ -152,7 +153,7 @@ public void testAuthenticatedProxyAsNonAdmin() throws Exception { } catch (PulsarAdminException.NotAuthorizedException e) { // expected } - adminAdmin.clusters().createCluster(configClusterName, new ClusterDataImpl(brokerUrl.toString())); + adminAdmin.clusters().createCluster(configClusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); adminAdmin.tenants().createTenant("tenant1", new TenantInfoImpl(ImmutableSet.of("unknownUser"), ImmutableSet.of(configClusterName))); diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java index c6798b1dc3918..ec5da713aafe8 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java @@ -39,6 +39,7 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.raw.RawMessageImpl; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.jctools.queues.SpscArrayQueue; @@ -61,7 +62,7 @@ public class TestCacheSizeAllocator extends MockedPulsarServiceBaseTest { protected void setup() throws Exception { super.internalSetup(); - admin.clusters().createCluster("test", new ClusterDataImpl(brokerUrl.toString())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); // so that clients can test short names admin.tenants().createTenant("public", diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java index 698ba19627aa2..5a0668eb9b0ac 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.testng.Assert; @@ -56,7 +57,7 @@ protected void setup() throws Exception { } }); // Setup namespaces - admin.clusters().createCluster("test", new ClusterDataImpl(pulsar.getWebServiceAddress())); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); TenantInfoImpl tenantInfo = new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test")); admin.tenants().createTenant(testTenant, tenantInfo); admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test")); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java index 199a4ca0206c4..1318e5c327530 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java @@ -42,7 +42,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -75,7 +75,7 @@ public class WebSocketService implements Closeable { private ServiceConfiguration config; private ConfigurationMetadataCacheService configurationCacheService; - private ClusterDataImpl localCluster; + private ClusterData localCluster; private final ConcurrentOpenHashMap> topicProducerMap; private final ConcurrentOpenHashMap> topicConsumerMap; private final ConcurrentOpenHashMap> topicReaderMap; @@ -85,7 +85,7 @@ public WebSocketService(WebSocketProxyConfiguration config) { this(createClusterData(config), PulsarConfigurationLoader.convertFrom(config)); } - public WebSocketService(ClusterDataImpl localCluster, ServiceConfiguration config) { + public WebSocketService(ClusterData localCluster, ServiceConfiguration config) { this.config = config; this.localCluster = localCluster; this.topicProducerMap = new ConcurrentOpenHashMap<>(); @@ -168,11 +168,11 @@ public synchronized PulsarClient getPulsarClient() throws IOException { return pulsarClient; } - public synchronized void setLocalCluster(ClusterDataImpl clusterData) { + public synchronized void setLocalCluster(ClusterData clusterData) { this.localCluster = clusterData; } - private PulsarClient createClientInstance(ClusterDataImpl clusterData) throws IOException { + private PulsarClient createClientInstance(ClusterData clusterData) throws IOException { ClientBuilder clientBuilder = PulsarClient.builder() // .statsInterval(0, TimeUnit.SECONDS) // .enableTls(config.isTlsEnabled()) // @@ -202,18 +202,25 @@ && isNotBlank(config.getBrokerClientAuthenticationParameters())) { return clientBuilder.build(); } - private static ClusterDataImpl createClusterData(WebSocketProxyConfiguration config) { + private static ClusterData createClusterData(WebSocketProxyConfiguration config) { if (isNotBlank(config.getBrokerServiceUrl()) || isNotBlank(config.getBrokerServiceUrlTls())) { - return new ClusterDataImpl(config.getServiceUrl(), config.getServiceUrlTls(), config.getBrokerServiceUrl(), - config.getBrokerServiceUrlTls()); + return ClusterData.builder() + .serviceUrl(config.getServiceUrl()) + .serviceUrlTls(config.getServiceUrlTls()) + .brokerServiceUrl(config.getBrokerServiceUrl()) + .brokerServiceUrlTls(config.getBrokerServiceUrlTls()) + .build(); } else if (isNotBlank(config.getServiceUrl()) || isNotBlank(config.getServiceUrlTls())) { - return new ClusterDataImpl(config.getServiceUrl(), config.getServiceUrlTls()); + return ClusterData.builder() + .serviceUrl(config.getServiceUrl()) + .serviceUrlTls(config.getServiceUrlTls()) + .build(); } else { return null; } } - private ClusterDataImpl retrieveClusterData() throws PulsarServerException { + private ClusterData retrieveClusterData() throws PulsarServerException { if (configurationCacheService == null) { throw new PulsarServerException( "Failed to retrieve Cluster data due to empty ConfigurationStoreServers"); diff --git a/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkBookieRackAffinityMappingTest.java b/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkBookieRackAffinityMappingTest.java index 9de2a74535bcf..3dde98bfdf9a3 100644 --- a/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkBookieRackAffinityMappingTest.java +++ b/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkBookieRackAffinityMappingTest.java @@ -123,8 +123,8 @@ public void testNoBookieInfo() throws Exception { Map> bookieMapping = new HashMap<>(); Map mainBookieGroup = new HashMap<>(); - mainBookieGroup.put(BOOKIE1, new BookieInfo("/rack0", null)); - mainBookieGroup.put(BOOKIE2, new BookieInfo("/rack1", null)); + mainBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("/rack0").build()); + mainBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("/rack1").build()); bookieMapping.put("group1", mainBookieGroup); @@ -146,8 +146,8 @@ public void testBookieInfoChange() throws Exception { Map> bookieMapping = new HashMap<>(); Map mainBookieGroup = new HashMap<>(); - mainBookieGroup.put(BOOKIE1, new BookieInfo("rack0", null)); - mainBookieGroup.put(BOOKIE2, new BookieInfo("rack1", null)); + mainBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + mainBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); bookieMapping.put("group1", mainBookieGroup); @@ -168,7 +168,7 @@ public void testBookieInfoChange() throws Exception { // add info for BOOKIE3 and check if the mapping picks up the change Map secondaryBookieGroup = new HashMap<>(); - secondaryBookieGroup.put(BOOKIE3, new BookieInfo("rack0", null)); + secondaryBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("group2", secondaryBookieGroup); localZkc.setData(ZkBookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), diff --git a/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicyTest.java b/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicyTest.java index 19ecd616d42f4..496a8583f8b0c 100644 --- a/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicyTest.java +++ b/pulsar-zookeeper-utils/src/test/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicyTest.java @@ -95,11 +95,11 @@ public void testBasic() throws Exception { Map> bookieMapping = new HashMap<>(); Map mainBookieGroup = new HashMap<>(); - mainBookieGroup.put(BOOKIE1, new BookieInfo("rack0", null)); - mainBookieGroup.put(BOOKIE2, new BookieInfo("rack1", null)); + mainBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + mainBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); Map secondaryBookieGroup = new HashMap<>(); - secondaryBookieGroup.put(BOOKIE3, new BookieInfo("rack0", null)); + secondaryBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("group1", mainBookieGroup); bookieMapping.put("group2", secondaryBookieGroup); @@ -138,7 +138,7 @@ public void testBasic() throws Exception { assertTrue(ensemble.contains(new BookieSocketAddress(BOOKIE4).toBookieId())); assertTrue(ensemble.contains(new BookieSocketAddress(BOOKIE2).toBookieId())); - secondaryBookieGroup.put(BOOKIE4, new BookieInfo("rack0", null)); + secondaryBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("group2", secondaryBookieGroup); localZkc.setData(ZkBookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), @@ -219,10 +219,10 @@ public void testBookieInfoChange() throws Exception { Map mainBookieGroup = new HashMap<>(); Map secondaryBookieGroup = new HashMap<>(); - mainBookieGroup.put(BOOKIE1, new BookieInfo("rack0", null)); - mainBookieGroup.put(BOOKIE2, new BookieInfo("rack1", null)); - secondaryBookieGroup.put(BOOKIE3, new BookieInfo("rack0", null)); - secondaryBookieGroup.put(BOOKIE4, new BookieInfo("rack2", null)); + mainBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + mainBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); + secondaryBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack0").build()); + secondaryBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack2").build()); bookieMapping.put("group1", mainBookieGroup); bookieMapping.put("group2", secondaryBookieGroup); @@ -251,7 +251,7 @@ public void testBookieInfoChange() throws Exception { // ok } - mainBookieGroup.put(BOOKIE3, new BookieInfo("rack1", null)); + mainBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack1").build()); secondaryBookieGroup.remove(BOOKIE3); bookieMapping.put("group1", mainBookieGroup); bookieMapping.put("group2", secondaryBookieGroup); @@ -315,16 +315,16 @@ public void testOverlappedBookies() throws Exception { Map defaultBookieGroup = new HashMap<>(); final String isolatedGroup = "isolatedGroup"; - defaultBookieGroup.put(BOOKIE1, new BookieInfo("rack0", null)); - defaultBookieGroup.put(BOOKIE2, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE3, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE4, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE5, new BookieInfo("rack1", null)); + defaultBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + defaultBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE5, BookieInfo.builder().rack("rack1").build()); Map isolatedBookieGroup = new HashMap<>(); - isolatedBookieGroup.put(BOOKIE1, new BookieInfo("rack1", null)); - isolatedBookieGroup.put(BOOKIE2, new BookieInfo("rack0", null)); - isolatedBookieGroup.put(BOOKIE4, new BookieInfo("rack0", null)); + isolatedBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack1").build()); + isolatedBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack0").build()); + isolatedBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("default", defaultBookieGroup); bookieMapping.put(isolatedGroup, isolatedBookieGroup); @@ -359,18 +359,18 @@ public void testSecondaryIsolationGroupsBookies() throws Exception { final String isolatedGroup = "primaryGroup"; final String secondaryIsolatedGroup = "secondaryGroup"; - defaultBookieGroup.put(BOOKIE1, new BookieInfo("rack0", null)); - defaultBookieGroup.put(BOOKIE2, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE3, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE4, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE5, new BookieInfo("rack1", null)); + defaultBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + defaultBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE5, BookieInfo.builder().rack("rack1").build()); Map primaryIsolatedBookieGroup = new HashMap<>(); - primaryIsolatedBookieGroup.put(BOOKIE1, new BookieInfo("rack1", null)); + primaryIsolatedBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack1").build()); Map secondaryIsolatedBookieGroup = new HashMap<>(); - secondaryIsolatedBookieGroup.put(BOOKIE2, new BookieInfo("rack0", null)); - secondaryIsolatedBookieGroup.put(BOOKIE4, new BookieInfo("rack0", null)); + secondaryIsolatedBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack0").build()); + secondaryIsolatedBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("default", defaultBookieGroup); bookieMapping.put(isolatedGroup, primaryIsolatedBookieGroup); @@ -408,14 +408,14 @@ public void testSecondaryIsolationGroupsBookiesNegative() throws Exception { final String isolatedGroup = "primaryGroup"; final String secondaryIsolatedGroup = "secondaryGroup"; - defaultBookieGroup.put(BOOKIE1, new BookieInfo("rack0", null)); - defaultBookieGroup.put(BOOKIE2, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE3, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE4, new BookieInfo("rack1", null)); - defaultBookieGroup.put(BOOKIE5, new BookieInfo("rack1", null)); + defaultBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + defaultBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack1").build()); + defaultBookieGroup.put(BOOKIE5, BookieInfo.builder().rack("rack1").build()); Map primaryIsolatedBookieGroup = new HashMap<>(); - primaryIsolatedBookieGroup.put(BOOKIE1, new BookieInfo("rack1", null)); + primaryIsolatedBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack1").build()); bookieMapping.put("default", defaultBookieGroup); bookieMapping.put(isolatedGroup, primaryIsolatedBookieGroup); @@ -461,12 +461,12 @@ public void testTheIsolationPolicyUsingCustomMetadata() throws Exception { Map primaryIsolationBookieGroups = new HashMap<>(); String primaryGroupName = "primary"; String secondaryGroupName = "secondary"; - primaryIsolationBookieGroups.put(BOOKIE1, new BookieInfo("rack0", null)); - primaryIsolationBookieGroups.put(BOOKIE2, new BookieInfo("rack0", null)); - primaryIsolationBookieGroups.put(BOOKIE3, new BookieInfo("rack1", null)); + primaryIsolationBookieGroups.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + primaryIsolationBookieGroups.put(BOOKIE2, BookieInfo.builder().rack("rack0").build()); + primaryIsolationBookieGroups.put(BOOKIE3, BookieInfo.builder().rack("rack1").build()); Map secondaryIsolationBookieGroups = new HashMap<>(); - secondaryIsolationBookieGroups.put(BOOKIE4, new BookieInfo("rack0", null)); + secondaryIsolationBookieGroups.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); bookieMapping.put(primaryGroupName, primaryIsolationBookieGroups); bookieMapping.put(secondaryGroupName, secondaryIsolationBookieGroups); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java index 453d4bcd567ec..87357a04cf6fb 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java @@ -27,7 +27,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.gson.Gson; import java.time.Duration; import java.util.Collections; import java.util.HashSet; @@ -67,6 +66,7 @@ import org.apache.pulsar.common.schema.KeyValueEncodingType; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.api.examples.AutoSchemaFunction; import org.apache.pulsar.functions.api.examples.AvroSchemaTestFunction; import org.apache.pulsar.functions.api.examples.MergeTopicFunction; @@ -161,13 +161,13 @@ public void testFunctionLocalRun(Runtime runtime) throws Exception { admin.topics().createNonPartitionedTopic(outputTopicName); retryStrategically((test) -> { try { - return admin.topics().getStats(inputTopicName).subscriptions.size() == 1; + return admin.topics().getStats(inputTopicName).getSubscriptions().size() == 1; } catch (PulsarAdminException e) { return false; } }, 30, 200); - assertEquals(admin.topics().getStats(inputTopicName).subscriptions.size(), 1); + assertEquals(admin.topics().getStats(inputTopicName).getSubscriptions().size(), 1); // publish and consume result if (Runtime.JAVA == runtime) { @@ -1055,8 +1055,8 @@ private void checkSubscriptionsCleanup(String topic) throws Exception { "topics", "stats", topic); - TopicStats topicStats = new Gson().fromJson(result.getStdout(), TopicStats.class); - assertEquals(topicStats.subscriptions.size(), 0); + TopicStats topicStats = ObjectMapperFactory.getThreadLocal().readValue(result.getStdout(), TopicStats.class); + assertEquals(topicStats.getSubscriptions().size(), 0); } catch (ContainerExecException e) { fail("Command should have exited with non-zero"); @@ -1070,8 +1070,8 @@ private void checkPublisherCleanup(String topic) throws Exception { "topics", "stats", topic); - TopicStats topicStats = new Gson().fromJson(result.getStdout(), TopicStats.class); - assertEquals(topicStats.publishers.size(), 0); + TopicStats topicStats = ObjectMapperFactory.getThreadLocal().readValue(result.getStdout(), TopicStats.class); + assertEquals(topicStats.getPublishers().size(), 0); } catch (ContainerExecException e) { fail("Command should have exited with non-zero"); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java index 38a5bb67348ee..0a755e2f6f6d2 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java @@ -83,7 +83,7 @@ private void testProxy(String serviceUrl, String httpServiceUrl) throws Exceptio for (int i = 0; i < 10; i++) { // Ensure we can get the stats for the topic irrespective of which broker the proxy decides to connect to TopicStats stats = admin.topics().getStats(topic); - assertEquals(stats.publishers.size(), 1); + assertEquals(stats.getPublishers().size(), 1); } } diff --git a/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java b/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java index 5153c56946529..8c3d3ce7c268e 100644 --- a/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java +++ b/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java @@ -29,7 +29,7 @@ import org.apache.pulsar.common.api.proto.SingleMessageMetadata; import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.shade.io.netty.buffer.ByteBuf; import org.apache.pulsar.shade.io.netty.buffer.Unpooled; @@ -75,7 +75,10 @@ public void setup() throws Exception { @Cleanup PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarContainer.getPulsarAdminUrl()).build(); admin.tenants().createTenant("my-property", - new TenantInfoImpl(new HashSet<>(Arrays.asList("appid1", "appid2")), Collections.singleton("standalone"))); + TenantInfo.builder() + .adminRoles(new HashSet<>(Arrays.asList("appid1", "appid2"))) + .allowedClusters(Collections.singleton("standalone")) + .build()); admin.namespaces().createNamespace("my-property/my-ns"); admin.namespaces().setNamespaceReplicationClusters("my-property/my-ns", Collections.singleton("standalone")); } diff --git a/tests/pulsar-client-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java b/tests/pulsar-client-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java index 51ce5f3e40edf..916a3bc6fe4d4 100644 --- a/tests/pulsar-client-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java +++ b/tests/pulsar-client-shade-test/src/test/java/org/apache/pulsar/tests/integration/SimpleProducerConsumerTest.java @@ -29,7 +29,7 @@ import org.apache.pulsar.common.api.proto.SingleMessageMetadata; import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.shade.io.netty.buffer.ByteBuf; import org.apache.pulsar.shade.io.netty.buffer.Unpooled; @@ -79,7 +79,8 @@ public void setup() throws Exception { PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(pulsarContainer.getPulsarAdminUrl()).build(); admin.tenants().createTenant("my-property", - new TenantInfoImpl(new HashSet<>(Arrays.asList("appid1", "appid2")), Collections.singleton("standalone"))); + TenantInfo.builder().adminRoles(new HashSet<>(Arrays.asList("appid1", "appid2"))) + .allowedClusters(Collections.singleton("standalone")).build()); admin.namespaces().createNamespace("my-property/my-ns"); admin.namespaces().setNamespaceReplicationClusters("my-property/my-ns", Collections.singleton("standalone")); admin.close();