From 89841aaf82b16ac4dfe7b7839211b3b38d6320a4 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 16 Sep 2021 08:27:36 -0700 Subject: [PATCH] Refactored PulsarResources to avoid building metadata paths everywhere (#11693) * Refactored PulsarResources to avoid building metadata paths everywhere * Fixed usages of ZkAdminPaths * Removed unused import * Fixed proxy * Fixed ResourceGroupServiceTest * Added MetadataStore deleteRecursive operation * Fixed merge issues * Use constant * Removed unused import * Fixes for ResourceGroupConfigListenerTest * Fixed cluster resources cleanup * Fixed topic list rest call * Fixed NPE in PersistentDispatcherFailoverConsumerTest setup * Fixed flaky test ReplicatorTest.testRemoveClusterFromNamespace * Fixed handling notification for V1 path namespaces * Fixed LocalPoliciesResources create * Fixed deadlock when making blocking calls from MetadataStore callback thread * Fixed usage of NamespaceName as key * Fixed missing mock in BacklogQuotaManagerConfigurationTest * Fixed RGUsageMTAggrWaitForAllMesgsTest * fixed ServerCnxTest * Fixed mock in PersistentSubscriptionTest * Fixed PersistentTopicTest * Fixed getting partitions list from NamespaceService * Fixed tenant not found expectation * Fixed ProxyConnection to check for existence of auth_data field --- .../PulsarAuthorizationProvider.java | 35 ++- .../broker/resources/BaseResources.java | 78 ++++-- .../broker/resources/BrokerResources.java | 42 +++ .../broker/resources/ClusterResources.java | 93 ++++++- .../DynamicConfigurationResources.java | 30 +++ .../resources/LocalPoliciesResources.java | 28 ++ .../broker/resources/NamespaceResources.java | 154 ++++++++++- .../resources/ResourceGroupResources.java | 55 ++++ .../broker/resources/TenantResources.java | 150 +++++++++++ .../broker/resources/TopicResources.java | 61 ++++- .../pulsar/PulsarClusterMetadataSetup.java | 27 +- .../apache/pulsar/broker/PulsarService.java | 11 +- .../pulsar/broker/admin/AdminResource.java | 87 +++---- .../pulsar/broker/admin/ZkAdminPaths.java | 47 ---- .../pulsar/broker/admin/impl/BrokersBase.java | 19 +- .../broker/admin/impl/ClustersBase.java | 111 +++----- .../broker/admin/impl/NamespacesBase.java | 243 +++++++----------- .../admin/impl/PersistentTopicsBase.java | 125 ++++----- .../broker/admin/impl/ResourceGroupsBase.java | 37 +-- .../pulsar/broker/admin/impl/TenantsBase.java | 45 ++-- .../pulsar/broker/admin/v1/Namespaces.java | 15 +- .../broker/admin/v2/NonPersistentTopics.java | 7 +- .../broker/namespace/NamespaceService.java | 42 ++- .../ResourceGroupConfigListener.java | 22 +- .../ResourceGroupNamespaceConfigListener.java | 56 ++-- .../resourcegroup/ResourceGroupService.java | 35 +-- .../broker/service/AbstractReplicator.java | 10 +- .../broker/service/BacklogQuotaManager.java | 26 +- .../pulsar/broker/service/BrokerService.java | 113 ++++---- .../nonpersistent/NonPersistentTopic.java | 10 +- .../persistent/MessageDeduplication.java | 11 +- .../service/persistent/PersistentTopic.java | 37 ++- .../pulsar/broker/web/PulsarWebResource.java | 181 ++----------- .../pulsar/broker/admin/AdminApiTest.java | 9 +- .../broker/admin/v1/V1_AdminApiTest.java | 4 +- .../broker/loadbalance/LoadBalancerTest.java | 14 +- .../SimpleLoadManagerImplTest.java | 6 +- .../RGUsageMTAggrWaitForAllMesgsTest.java | 161 ++++++------ .../ResourceGroupConfigListenerTest.java | 21 +- .../ResourceGroupRateLimiterTest.java | 4 +- .../ResourceGroupServiceTest.java | 10 +- .../ResourceGroupUsageAggregationTest.java | 7 +- .../BacklogQuotaManagerConfigurationTest.java | 2 + .../service/BrokerServiceThrottlingTest.java | 16 +- ...sistentDispatcherFailoverConsumerTest.java | 4 + .../broker/service/PersistentTopicTest.java | 33 ++- .../service/ReplicatorRemoveClusterTest.java | 110 ++++++++ .../pulsar/broker/service/ReplicatorTest.java | 49 ---- .../pulsar/broker/service/ServerCnxTest.java | 6 +- .../PersistentSubscriptionTest.java | 2 + .../pulsar/broker/web/WebServiceTest.java | 3 +- .../zookeeper/ClusterMetadataSetupTest.java | 25 +- .../client/api/PartitionCreationTest.java | 5 +- .../impl/BrokerClientIntegrationTest.java | 14 +- .../service/BrokerDiscoveryProvider.java | 19 +- .../functions/worker/PulsarWorkerService.java | 17 +- .../metadata/api/MetadataStoreException.java | 4 + .../cache/impl/MetadataCacheImpl.java | 3 +- .../proxy/server/BrokerDiscoveryProvider.java | 24 +- .../pulsar/websocket/WebSocketService.java | 5 +- 60 files changed, 1412 insertions(+), 1208 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BrokerResources.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ZkAdminPaths.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRemoveClusterTest.java 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 178cb1cf5ab9f..c713fe2791fbb 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 @@ -58,9 +58,7 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider { public ServiceConfiguration conf; private PulsarResources pulsarResources; - private static final String POLICY_ROOT = "/admin/policies/"; - public static final String POLICIES = "policies"; - private static final String POLICIES_READONLY_FLAG_PATH = "/admin/flags/policies-readonly"; + public PulsarAuthorizationProvider() { } @@ -109,7 +107,8 @@ public CompletableFuture canConsumeAsync(TopicName topicName, String ro AuthenticationDataSource authenticationData, String subscription) { CompletableFuture permissionFuture = new CompletableFuture<>(); try { - pulsarResources.getNamespaceResources().getAsync(POLICY_ROOT + topicName.getNamespace()).thenAccept(policies -> { + pulsarResources.getNamespaceResources().getPoliciesAsync(topicName.getNamespaceObject()) + .thenAccept(policies -> { if (!policies.isPresent()) { if (log.isDebugEnabled()) { log.debug("Policies node couldn't be found for topic : {}", topicName); @@ -229,7 +228,7 @@ private CompletableFuture allowTheSpecifiedActionOpsAsync(NamespaceName AuthAction authAction) { CompletableFuture permissionFuture = new CompletableFuture<>(); try { - pulsarResources.getNamespaceResources().getAsync(POLICY_ROOT + namespaceName.toString()).thenAccept(policies -> { + pulsarResources.getNamespaceResources().getPoliciesAsync(namespaceName).thenAccept(policies -> { if (!policies.isPresent()) { if (log.isDebugEnabled()) { log.debug("Policies node couldn't be found for namespace : {}", namespaceName); @@ -285,9 +284,8 @@ public CompletableFuture grantPermissionAsync(NamespaceName namespaceName, result.completeExceptionally(e); } - final String policiesPath = String.format("/%s/%s/%s", "admin", POLICIES, namespaceName.toString()); try { - pulsarResources.getNamespaceResources().set(policiesPath, (policies)->{ + pulsarResources.getNamespaceResources().setPolicies(namespaceName, policies -> { policies.auth_policies.getNamespaceAuthentication().put(role, actions); return policies; }); @@ -300,7 +298,7 @@ public CompletableFuture grantPermissionAsync(NamespaceName namespaceName, } catch (BadVersionException e) { log.warn("[{}] Failed to set permissions for namespace {}: concurrent modification", role, namespaceName); result.completeExceptionally(new IllegalStateException( - "Concurrent modification on zk path: " + policiesPath + ", " + e.getMessage())); + "Concurrent modification on metadata: " + namespaceName + ", " + e.getMessage())); } catch (Exception e) { log.error("[{}] Failed to get permissions for namespace {}", role, namespaceName, e); result.completeExceptionally( @@ -332,11 +330,9 @@ private CompletableFuture updateSubscriptionPermissionAsync(NamespaceName result.completeExceptionally(e); } - final String policiesPath = String.format("/%s/%s/%s", "admin", POLICIES, namespace.toString()); - try { - Policies policies = pulsarResources.getNamespaceResources().get(policiesPath) - .orElseThrow(() -> new NotFoundException(policiesPath + " not found")); + Policies policies = pulsarResources.getNamespaceResources().getPolicies(namespace) + .orElseThrow(() -> new NotFoundException(namespace + " not found")); if (remove) { if (policies.auth_policies.getSubscriptionAuthentication().get(subscriptionName) != null) { policies.auth_policies.getSubscriptionAuthentication().get(subscriptionName).removeAll(roles); @@ -348,7 +344,7 @@ private CompletableFuture updateSubscriptionPermissionAsync(NamespaceName } else { policies.auth_policies.getSubscriptionAuthentication().put(subscriptionName, roles); } - pulsarResources.getNamespaceResources().set(policiesPath, (data)->policies); + pulsarResources.getNamespaceResources().setPolicies(namespace, (data)->policies); log.info("[{}] Successfully granted access for role {} for sub = {}", namespace, subscriptionName, roles); result.complete(null); @@ -358,7 +354,7 @@ private CompletableFuture updateSubscriptionPermissionAsync(NamespaceName } catch (BadVersionException e) { log.warn("[{}] Failed to set permissions for {} on namespace {}: concurrent modification", subscriptionName, roles, namespace); result.completeExceptionally(new IllegalStateException( - "Concurrent modification on zk path: " + policiesPath + ", " + e.getMessage())); + "Concurrent modification on metadata path: " + namespace + ", " + e.getMessage())); } catch (Exception e) { log.error("[{}] Failed to get permissions for role {} on namespace {}", subscriptionName, roles, namespace, e); result.completeExceptionally( @@ -388,7 +384,8 @@ private boolean checkCluster(TopicName topicName) { public CompletableFuture checkPermission(TopicName topicName, String role, AuthAction action) { CompletableFuture permissionFuture = new CompletableFuture<>(); try { - pulsarResources.getNamespaceResources().getAsync(POLICY_ROOT + topicName.getNamespace()).thenAccept(policies -> { + pulsarResources.getNamespaceResources().getPoliciesAsync(topicName.getNamespaceObject()) + .thenAccept(policies -> { if (!policies.isPresent()) { if (log.isDebugEnabled()) { log.debug("Policies node couldn't be found for topic : {}", topicName); @@ -497,10 +494,10 @@ private void validatePoliciesReadOnlyAccess() { boolean arePoliciesReadOnly = true; try { - arePoliciesReadOnly = pulsarResources.getNamespaceResources().exists(POLICIES_READONLY_FLAG_PATH); + arePoliciesReadOnly = pulsarResources.getNamespaceResources().getPoliciesReadOnly(); } catch (Exception e) { - log.warn("Unable to fetch contents of [{}] from global zookeeper", POLICIES_READONLY_FLAG_PATH, e); - throw new IllegalStateException("Unable to fetch content from global zk"); + log.warn("Unable to check if policies are read-only", e); + throw new IllegalStateException("Unable to fetch content from configuration metadata store"); } if (arePoliciesReadOnly) { @@ -632,7 +629,7 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, } else { try { TenantInfo tenantInfo = pulsarResources.getTenantResources() - .get(path(POLICIES, tenantName)) + .getTenant(tenantName) .orElseThrow(() -> new RestException(Response.Status.NOT_FOUND, "Tenant does not exist")); return isTenantAdmin(tenantName, role, tenantInfo, authData); } catch (NotFoundException e) { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java index 9f756fcce01dd..9061dd76b47fd 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java @@ -19,6 +19,10 @@ package org.apache.pulsar.broker.resources; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Joiner; +import java.util.ArrayList; +import java.util.Deque; +import java.util.LinkedList; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -26,9 +30,11 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import lombok.Getter; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.zookeeper.common.PathUtils; /** * Base class for all configuration resources to access configurations from metadata-store. @@ -36,8 +42,12 @@ * @param * type of configuration-resources. */ +@Slf4j public class BaseResources { + protected static final String BASE_POLICIES_PATH = "/admin/policies"; + protected static final String BASE_CLUSTERS_PATH = "/admin/clusters"; + @Getter private final MetadataStore store; @Getter @@ -56,7 +66,7 @@ public BaseResources(MetadataStore store, TypeReference typeRef, int operatio this.operationTimeoutSec = operationTimeoutSec; } - public List getChildren(String path) throws MetadataStoreException { + protected List getChildren(String path) throws MetadataStoreException { try { return getChildrenAsync(path).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { @@ -67,11 +77,11 @@ public List getChildren(String path) throws MetadataStoreException { } } - public CompletableFuture> getChildrenAsync(String path) { + protected CompletableFuture> getChildrenAsync(String path) { return cache.getChildren(path); } - public Optional get(String path) throws MetadataStoreException { + protected Optional get(String path) throws MetadataStoreException { try { return getAsync(path).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { @@ -82,11 +92,11 @@ public Optional get(String path) throws MetadataStoreException { } } - public CompletableFuture> getAsync(String path) { + protected CompletableFuture> getAsync(String path) { return cache.get(path); } - public void set(String path, Function modifyFunction) throws MetadataStoreException { + protected void set(String path, Function modifyFunction) throws MetadataStoreException { try { setAsync(path, modifyFunction).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { @@ -97,11 +107,11 @@ public void set(String path, Function modifyFunction) throws MetadataStore } } - public CompletableFuture setAsync(String path, Function modifyFunction) { + protected CompletableFuture setAsync(String path, Function modifyFunction) { return cache.readModifyUpdate(path, modifyFunction).thenApply(__ -> null); } - public void setWithCreate(String path, Function, T> createFunction) throws MetadataStoreException { + protected void setWithCreate(String path, Function, T> createFunction) throws MetadataStoreException { try { setWithCreateAsync(path, createFunction).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { @@ -112,11 +122,11 @@ public void setWithCreate(String path, Function, T> createFunction) } } - public CompletableFuture setWithCreateAsync(String path, Function, T> createFunction) { + protected CompletableFuture setWithCreateAsync(String path, Function, T> createFunction) { return cache.readModifyUpdateOrCreate(path, createFunction).thenApply(__ -> null); } - public void create(String path, T data) throws MetadataStoreException { + protected void create(String path, T data) throws MetadataStoreException { try { createAsync(path, data).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { @@ -127,11 +137,11 @@ public void create(String path, T data) throws MetadataStoreException { } } - public CompletableFuture createAsync(String path, T data) { + protected CompletableFuture createAsync(String path, T data) { return cache.create(path, data); } - public void delete(String path) throws MetadataStoreException { + protected void delete(String path) throws MetadataStoreException { try { deleteAsync(path).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { @@ -142,13 +152,13 @@ public void delete(String path) throws MetadataStoreException { } } - public CompletableFuture deleteAsync(String path) { + protected CompletableFuture deleteAsync(String path) { return cache.delete(path); } - public boolean exists(String path) throws MetadataStoreException { + protected boolean exists(String path) throws MetadataStoreException { try { - return existsAsync(path).get(operationTimeoutSec, TimeUnit.SECONDS); + return cache.exists(path).get(operationTimeoutSec, TimeUnit.SECONDS); } catch (ExecutionException e) { throw (e.getCause() instanceof MetadataStoreException) ? (MetadataStoreException) e.getCause() : new MetadataStoreException(e.getCause()); @@ -161,7 +171,43 @@ public int getOperationTimeoutSec() { return operationTimeoutSec; } - public CompletableFuture existsAsync(String path) { - return cache.exists(path); + protected static String joinPath(String... parts) { + StringBuilder sb = new StringBuilder(); + Joiner.on('/').appendTo(sb, parts); + return sb.toString(); + } + + + + protected static void deleteRecursive(BaseResources resources, final String pathRoot) throws MetadataStoreException { + PathUtils.validatePath(pathRoot); + List tree = listSubTreeBFS(resources, pathRoot); + log.debug("Deleting {} with size {}", tree, tree.size()); + log.debug("Deleting " + tree.size() + " subnodes "); + for (int i = tree.size() - 1; i >= 0; --i) { + // Delete the leaves first and eventually get rid of the root + resources.delete(tree.get(i)); + } + } + + protected static List listSubTreeBFS(BaseResources resources, final String pathRoot) + throws MetadataStoreException { + Deque queue = new LinkedList<>(); + List tree = new ArrayList<>(); + queue.add(pathRoot); + tree.add(pathRoot); + while (true) { + String node = queue.pollFirst(); + if (node == null) { + break; + } + List children = resources.getChildren(node); + for (final String child : children) { + final String childPath = node + "/" + child; + queue.add(childPath); + tree.add(childPath); + } + } + return tree; } } \ No newline at end of file diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BrokerResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BrokerResources.java new file mode 100644 index 0000000000000..4aca5d042c340 --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BrokerResources.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.resources; + +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.FailureDomainImpl; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; + +public class BrokerResources extends BaseResources { + + public BrokerResources(MetadataStore store, int operationTimeoutSec) { + super(store, ClusterData.class, operationTimeoutSec); + } + + public Set listActiveBrokers() throws MetadataStoreException { + return new HashSet<>(super.getChildren(BASE_CLUSTERS_PATH)); + } +} 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 c5bfde772611d..af3d8c3c7e330 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 @@ -18,17 +18,25 @@ */ package org.apache.pulsar.broker.resources; +import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import java.util.HashSet; +import java.util.List; +import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; import lombok.Getter; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.cache.ConfigurationCacheService; +import org.apache.pulsar.common.naming.Metadata; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; public class ClusterResources extends BaseResources { - public static final String CLUSTERS_ROOT = "/admin/clusters"; @Getter private FailureDomainResources failureDomainResources; @@ -38,7 +46,55 @@ public ClusterResources(MetadataStore store, int operationTimeoutSec) { } public Set list() throws MetadataStoreException { - return new HashSet<>(super.getChildren(CLUSTERS_ROOT)); + return new HashSet<>(super.getChildren(BASE_CLUSTERS_PATH)); + } + + public Optional getCluster(String clusterName) throws MetadataStoreException { + return get(joinPath(BASE_CLUSTERS_PATH, clusterName)); + } + + public List getNamespacesForCluster(String tenant, String clusterName) throws MetadataStoreException { + return getChildren(joinPath(BASE_POLICIES_PATH, tenant, clusterName)); + } + + public void createCluster(String clusterName, ClusterData clusterData) throws MetadataStoreException { + create(joinPath(BASE_CLUSTERS_PATH, clusterName), clusterData); + } + + public void updateCluster(String clusterName, Function modifyFunction) + throws MetadataStoreException { + set(joinPath(BASE_CLUSTERS_PATH, clusterName), modifyFunction); + } + + public void deleteCluster(String clusterName) throws MetadataStoreException { + delete(joinPath(BASE_CLUSTERS_PATH, clusterName)); + } + + public boolean isClusterUsed(String clusterName) throws MetadataStoreException { + for (String tenant : getCache().getChildren(BASE_POLICIES_PATH).join()) { + if (!getCache().getChildren(joinPath(BASE_POLICIES_PATH, tenant, clusterName)).join().isEmpty()) { + // We found a tenant that has at least a namespace in this cluster + return true; + } + } + + return false; + } + + public boolean clusterExists(String clusterName) throws MetadataStoreException { + return exists(joinPath(BASE_CLUSTERS_PATH, clusterName)); + } + + public CompletableFuture clusterExistsAsync(String clusterName) { + return getCache().exists(joinPath(BASE_CLUSTERS_PATH, clusterName)); + } + + public static boolean pathRepresentsClusterName(String path) { + return path.startsWith(BASE_CLUSTERS_PATH); + } + + public static String clusterNameFromPath(String path) { + return path.substring(BASE_CLUSTERS_PATH.length() + 1); } public static class FailureDomainResources extends BaseResources { @@ -48,5 +104,38 @@ public FailureDomainResources(MetadataStore store, Class claz int operationTimeoutSec) { super(store, clazz, operationTimeoutSec); } + + public List listFailureDomains(String clusterName) throws MetadataStoreException { + return getChildren(joinPath(BASE_CLUSTERS_PATH, clusterName, FAILURE_DOMAIN)); + } + + public Optional getFailureDomain(String clusterName, String domainName) + throws MetadataStoreException { + return get(joinPath(BASE_CLUSTERS_PATH, clusterName, FAILURE_DOMAIN, domainName)); + } + + public void deleteFailureDomain(String clusterName, String domainName) throws MetadataStoreException { + String path = joinPath(BASE_CLUSTERS_PATH, clusterName, FAILURE_DOMAIN, domainName); + if (exists(path)) { + delete(path); + } + } + + public void deleteFailureDomains(String clusterName) throws MetadataStoreException { + String failureDomainPath = joinPath(BASE_CLUSTERS_PATH, clusterName, FAILURE_DOMAIN); + for (String domain : getChildren(failureDomainPath)) { + delete(joinPath(failureDomainPath, domain)); + } + + if (exists(failureDomainPath)) { + delete(failureDomainPath); + } + } + + public void setFailureDomainWithCreate(String clusterName, String domainName, + Function, FailureDomainImpl> createFunction) + throws MetadataStoreException { + setWithCreate(joinPath(BASE_CLUSTERS_PATH, clusterName, FAILURE_DOMAIN, domainName), createFunction); + } } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java index ce6e7979947ec..8137dd837ef96 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java @@ -19,14 +19,44 @@ package org.apache.pulsar.broker.resources; import com.fasterxml.jackson.core.type.TypeReference; +import java.util.Collections; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; public class DynamicConfigurationResources extends BaseResources> { + private static final String BROKER_SERVICE_CONFIGURATION_PATH = "/admin/configuration"; + public DynamicConfigurationResources(MetadataStore store, int operationTimeoutSec) { super(store, new TypeReference>() { }, operationTimeoutSec); } + public CompletableFuture>> getDynamicConfigurationAsync() { + return getAsync(BROKER_SERVICE_CONFIGURATION_PATH); + } + + public Map getDynamicConfiguration() throws MetadataStoreException { + return get(BROKER_SERVICE_CONFIGURATION_PATH).orElse(Collections.emptyMap()); + } + + public void setDynamicConfigurationWithCreate( + Function>, Map> createFunction) + throws MetadataStoreException { + super.setWithCreate(BROKER_SERVICE_CONFIGURATION_PATH, createFunction); + } + + public void setDynamicConfiguration( + Function, Map> updateFunction) + throws MetadataStoreException { + super.set(BROKER_SERVICE_CONFIGURATION_PATH, updateFunction); + } + + public boolean isDynamicConfigurationPath(String path) { + return BROKER_SERVICE_CONFIGURATION_PATH.equals(path); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java index dda042b527f19..9e2f1052f44e6 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java @@ -18,12 +18,40 @@ */ package org.apache.pulsar.broker.resources; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.LocalPolicies; import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; public class LocalPoliciesResources extends BaseResources { + private static final String LOCAL_POLICIES_ROOT = "/admin/local-policies"; + public LocalPoliciesResources(MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, LocalPolicies.class, operationTimeoutSec); } + + public void setLocalPolicies(NamespaceName ns, Function modifyFunction) + throws MetadataStoreException { + set(joinPath(LOCAL_POLICIES_ROOT, ns.toString()), modifyFunction); + } + + public Optional getLocalPolicies(NamespaceName ns) throws MetadataStoreException{ + return get(joinPath(LOCAL_POLICIES_ROOT, ns.toString())); + } + + public CompletableFuture> getLocalPoliciesAsync(NamespaceName ns) { + return getCache().get(joinPath(LOCAL_POLICIES_ROOT, ns.toString())); + } + + public void setLocalPoliciesWithCreate(NamespaceName ns, Function, LocalPolicies> createFunction) throws MetadataStoreException { + setWithCreate(joinPath(LOCAL_POLICIES_ROOT, ns.toString()), createFunction); + } + + public void deleteLocalPolicies(NamespaceName ns) throws MetadataStoreException { + delete(joinPath(LOCAL_POLICIES_ROOT, ns.toString())); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java index 1698bd321af9c..395b8e669f652 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java @@ -18,20 +18,27 @@ */ package org.apache.pulsar.broker.resources; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.apache.pulsar.common.policies.path.PolicyPath.path; import com.fasterxml.jackson.core.type.TypeReference; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; import lombok.Getter; 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.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.LocalPolicies; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; -import org.apache.pulsar.common.policies.path.PolicyPath; +import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStore; @@ -45,7 +52,7 @@ public class NamespaceResources extends BaseResources { private final MetadataCache localPoliciesCache; - private static final String LOCAL_POLICIES_ROOT = "/admin/local-policies"; + private static final String POLICIES_READONLY_FLAG_PATH = "/admin/flags/policies-readonly"; public NamespaceResources(MetadataStore localStore, MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, Policies.class, operationTimeoutSec); @@ -60,33 +67,154 @@ public NamespaceResources(MetadataStore localStore, MetadataStore configurationS } } - public CompletableFuture> getPolicies(NamespaceName ns) { - return getCache().get(PolicyPath.path(POLICIES, ns.toString())); + public CompletableFuture> listNamespacesAsync(String tenant) { + return getChildrenAsync(joinPath(BASE_POLICIES_PATH, tenant)); } - public CompletableFuture> getLocalPolicies(NamespaceName ns) { - if (localPoliciesCache == null) { - return FutureUtil.failedFuture(new IllegalStateException("Local metadata store not setup")); - } else { - return localPoliciesCache.get(PolicyPath.joinPath(LOCAL_POLICIES_ROOT, ns.toString())); - } + public boolean getPoliciesReadOnly() throws MetadataStoreException { + return super.exists(POLICIES_READONLY_FLAG_PATH); + } + + public void createPolicies(NamespaceName ns, Policies policies) throws MetadataStoreException{ + create(joinPath(BASE_POLICIES_PATH, ns.toString()), policies); + } + + public boolean namespaceExists(NamespaceName ns) throws MetadataStoreException { + String path = joinPath(BASE_POLICIES_PATH, ns.toString()); + return super.exists(path) && + super.getChildren(path).isEmpty(); + } + + public CompletableFuture namespaceExistsAsync(NamespaceName ns) { + String path = joinPath(BASE_POLICIES_PATH, ns.toString()); + return getCache().exists(path) + .thenCompose(exists -> { + if (!exists) { + return CompletableFuture.completedFuture(false); + } else { + return getChildrenAsync(path).thenApply(children -> children.isEmpty()); + } + }); + } + + public void deletePolicies(NamespaceName ns) throws MetadataStoreException{ + delete(joinPath(BASE_POLICIES_PATH, ns.toString())); + } + + public Optional getPolicies(NamespaceName ns) throws MetadataStoreException{ + return get(joinPath(BASE_POLICIES_PATH, ns.toString())); + } + + public CompletableFuture> getPoliciesAsync(NamespaceName ns) { + return getCache().get(joinPath(BASE_POLICIES_PATH, ns.toString())); + } + + public void setPolicies(NamespaceName ns, Function function) throws MetadataStoreException { + set(joinPath(BASE_POLICIES_PATH, ns.toString()), function); + } + + public CompletableFuture setPoliciesAsync(NamespaceName ns, Function function) { + return setAsync(joinPath(BASE_POLICIES_PATH, ns.toString()), function); + } + + public static boolean pathIsFromNamespace(String path) { + return path.startsWith(BASE_POLICIES_PATH); + } + + public static NamespaceName namespaceFromPath(String path) { + return NamespaceName.get(path.substring(BASE_POLICIES_PATH.length() + 1)); } public static class IsolationPolicyResources extends BaseResources> { + private static final String NAMESPACE_ISOLATION_POLICIES = "namespaceIsolationPolicies"; + public IsolationPolicyResources(MetadataStore store, int operationTimeoutSec) { super(store, new TypeReference>() { }, operationTimeoutSec); } - public Optional getPolicies(String path) throws MetadataStoreException { - Optional> data = super.get(path); + public Optional getIsolationDataPolicies(String cluster) + throws MetadataStoreException { + Optional> data = + super.get(joinPath(BASE_CLUSTERS_PATH, cluster, NAMESPACE_ISOLATION_POLICIES)); return data.isPresent() ? Optional.of(new NamespaceIsolationPolicies(data.get())) : Optional.empty(); } + + public void deleteIsolationData(String cluster) throws MetadataStoreException { + delete(joinPath(BASE_CLUSTERS_PATH, cluster, NAMESPACE_ISOLATION_POLICIES)); + } + + public void createIsolationData(String cluster, Map id) + throws MetadataStoreException { + create(joinPath(BASE_CLUSTERS_PATH, cluster, NAMESPACE_ISOLATION_POLICIES), id); + } + + public void setIsolationData(String cluster, + Function, Map> modifyFunction) + throws MetadataStoreException { + set(joinPath(BASE_CLUSTERS_PATH, cluster, NAMESPACE_ISOLATION_POLICIES), modifyFunction); + } + + public void setIsolationDataWithCreate(String cluster, + Function>, Map> createFunction) + throws MetadataStoreException { + setWithCreate(joinPath(BASE_CLUSTERS_PATH, cluster, NAMESPACE_ISOLATION_POLICIES), createFunction); + } } public static class PartitionedTopicResources extends BaseResources { + private static final String PARTITIONED_TOPIC_PATH = "/admin/partitioned-topics"; + public PartitionedTopicResources(MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, PartitionedTopicMetadata.class, operationTimeoutSec); } + + public CompletableFuture updatePartitionedTopicAsync(TopicName tn, Function f) { + return setAsync(joinPath(PARTITIONED_TOPIC_PATH, tn.getNamespace(), tn.getDomain().value(), + tn.getEncodedLocalName()), f); + } + + public void createPartitionedTopic(TopicName tn, PartitionedTopicMetadata tm) throws MetadataStoreException { + create(joinPath(PARTITIONED_TOPIC_PATH, tn.getNamespace(), tn.getDomain().value(), + tn.getEncodedLocalName()), tm); + } + + public CompletableFuture createPartitionedTopicAsync(TopicName tn, PartitionedTopicMetadata tm) { + return createAsync(joinPath(PARTITIONED_TOPIC_PATH, tn.getNamespace(), tn.getDomain().value(), + tn.getEncodedLocalName()), tm); + } + + public CompletableFuture> listPartitionedTopicsAsync(NamespaceName ns, TopicDomain domain) { + return getChildrenAsync(joinPath(PARTITIONED_TOPIC_PATH, ns.toString(), domain.value())) + .thenApply(list -> + list.stream().map(x -> TopicName.get(domain.value(), ns, Codec.decode(x)).toString()) + .collect(Collectors.toList()) + ); + } + + public CompletableFuture> getPartitionedTopicMetadataAsync(TopicName tn) { + return getAsync(joinPath(PARTITIONED_TOPIC_PATH, tn.getNamespace(), tn.getDomain().value(), + tn.getEncodedLocalName())); + } + + public boolean partitionedTopicExists(TopicName tn) throws MetadataStoreException { + return exists(joinPath(PARTITIONED_TOPIC_PATH, tn.getNamespace(), tn.getDomain().value(), + tn.getEncodedLocalName())); + } + + public CompletableFuture deletePartitionedTopicAsync(TopicName tn) { + return deleteAsync(joinPath(PARTITIONED_TOPIC_PATH, tn.getNamespace(), tn.getDomain().value(), + tn.getEncodedLocalName())); + } + + public void clearPartitionedTopicMetadata(NamespaceName namespaceName) throws MetadataStoreException { + final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString()); + // check whether partitioned topics metadata node exist + if (exists(globalPartitionedPath)) { + deleteRecursive(this, globalPartitionedPath); + } + } } } \ No newline at end of file diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java index 643b00355fb22..a0b19d6ff24a6 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java @@ -18,11 +18,66 @@ */ package org.apache.pulsar.broker.resources; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.ResourceGroup; import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; public class ResourceGroupResources extends BaseResources { + + private static final String BASE_PATH = "/admin/resourcegroups"; + public ResourceGroupResources(MetadataStore store, int operationTimeoutSec) { super(store, ResourceGroup.class, operationTimeoutSec); } + + public Optional getResourceGroup(String resourceGroupName) throws MetadataStoreException { + return get(joinPath(BASE_PATH, resourceGroupName)); + } + + public CompletableFuture> getResourceGroupAsync(String resourceGroupName) { + return getAsync(joinPath(BASE_PATH, resourceGroupName)); + } + + public boolean resourceGroupExists(String resourceGroupName) throws MetadataStoreException { + return exists(joinPath(BASE_PATH, resourceGroupName)); + } + + public void createResourceGroup(String resourceGroupName, ResourceGroup rg) throws MetadataStoreException { + create(joinPath(BASE_PATH, resourceGroupName), rg); + } + + public void deleteResourceGroup(String resourceGroupName) throws MetadataStoreException { + delete(joinPath(BASE_PATH, resourceGroupName)); + } + + public void updateResourceGroup(String resourceGroupName, + Function modifyFunction) + throws MetadataStoreException { + set(joinPath(BASE_PATH, resourceGroupName), modifyFunction); + } + + public List listResourceGroups() throws MetadataStoreException { + return getChildren(BASE_PATH); + } + + public CompletableFuture> listResourceGroupsAsync(){ + return getChildrenAsync(BASE_PATH); + } + + public static boolean isResourceGroupPath(String path) { + return path.startsWith(BASE_PATH); + } + + public static Optional resourceGroupNameFromPath(String path) { + if (path.length() > BASE_PATH.length() + 1) { + return Optional.of(path.substring(BASE_PATH.length() + 1)); + } else { + return Optional.empty(); + } + } } 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 86b0397b6a023..524c6aa353e13 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,161 @@ */ package org.apache.pulsar.broker.resources; +import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; +import javax.ws.rs.core.Response; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; +@Slf4j public class TenantResources extends BaseResources { public TenantResources(MetadataStore store, int operationTimeoutSec) { super(store, TenantInfo.class, operationTimeoutSec); } + + public List listTenants() throws MetadataStoreException { + return getChildren(BASE_POLICIES_PATH); + } + + public CompletableFuture> listTenantsAsync() { + return getChildrenAsync(BASE_POLICIES_PATH); + } + + public CompletableFuture deleteTenantAsync(String tenantName) { + return getChildrenAsync(joinPath(BASE_POLICIES_PATH, tenantName)) + .thenCompose(clusters -> FutureUtil.waitForAll(clusters.stream() + .map(cluster -> getCache().delete(joinPath(BASE_POLICIES_PATH, tenantName, cluster))) + .collect(Collectors.toList())) + ).thenCompose(__ -> deleteAsync(joinPath(BASE_POLICIES_PATH, tenantName))); + } + + public boolean tenantExists(String tenantName) throws MetadataStoreException { + return exists(joinPath(BASE_POLICIES_PATH, tenantName)); + } + + public Optional getTenant(String tenantName) throws MetadataStoreException { + return get(joinPath(BASE_POLICIES_PATH, tenantName)); + } + + public CompletableFuture> getTenantAsync(String tenantName) { + return getAsync(joinPath(BASE_POLICIES_PATH, tenantName)); + } + + public void createTenant(String tenantName, TenantInfo ti) throws MetadataStoreException { + create(joinPath(BASE_POLICIES_PATH, tenantName), ti); + } + + public CompletableFuture createTenantAsync(String tenantName, TenantInfo ti) { + return createAsync(joinPath(BASE_POLICIES_PATH, tenantName), ti); + } + + public CompletableFuture updateTenantAsync(String tenantName, Function f) { + return setAsync(joinPath(BASE_POLICIES_PATH, tenantName), f); + } + + public CompletableFuture tenantExistsAsync(String tenantName) { + return getCache().exists(joinPath(BASE_POLICIES_PATH, tenantName)); + } + + public List getListOfNamespaces(String tenant) throws MetadataStoreException { + List namespaces = Lists.newArrayList(); + + // this will return a cluster in v1 and a namespace in v2 + for (String clusterOrNamespace : getChildren(joinPath(BASE_POLICIES_PATH, tenant))) { + // Then get the list of namespaces + final List children = getChildren(joinPath(BASE_POLICIES_PATH, tenant, clusterOrNamespace)); + if (children == null || children.isEmpty()) { + String namespace = NamespaceName.get(tenant, clusterOrNamespace).toString(); + // if the length is 0 then this is probably a leftover cluster from namespace created + // with the v1 admin format (prop/cluster/ns) and then deleted, so no need to add it to the list + try { + if (get(joinPath(BASE_POLICIES_PATH, namespace)).isPresent()) { + namespaces.add(namespace); + } + } catch (MetadataStoreException.ContentDeserializationException e) { + // not a namespace node + } + + } else { + children.forEach(ns -> { + namespaces.add(NamespaceName.get(tenant, clusterOrNamespace, ns).toString()); + }); + } + } + + return namespaces; + } + + public CompletableFuture> getActiveNamespaces(String tenant, String cluster) { + return getChildrenAsync(joinPath(BASE_POLICIES_PATH, tenant, cluster)); + } + + public CompletableFuture hasActiveNamespace(String tenant) { + CompletableFuture activeNamespaceFuture = new CompletableFuture<>(); + getChildrenAsync(joinPath(BASE_POLICIES_PATH, tenant)).thenAccept(clusterOrNamespaceList -> { + if (clusterOrNamespaceList == null || clusterOrNamespaceList.isEmpty()) { + activeNamespaceFuture.complete(null); + return; + } + List> activeNamespaceListFuture = Lists.newArrayList(); + clusterOrNamespaceList.forEach(clusterOrNamespace -> { + // get list of active V1 namespace + CompletableFuture checkNs = new CompletableFuture<>(); + activeNamespaceListFuture.add(checkNs); + getChildrenAsync(joinPath(BASE_POLICIES_PATH, tenant, clusterOrNamespace)) + .whenComplete((children, ex) -> { + if (ex != null) { + checkNs.completeExceptionally(ex); + return; + } + if (children != null && !children.isEmpty()) { + checkNs.completeExceptionally( + new IllegalStateException("Tenant has active namespace")); + return; + } + String namespace = NamespaceName.get(tenant, clusterOrNamespace).toString(); + // if the length is 0 then this is probably a leftover cluster from namespace + // created + // with the v1 admin format (prop/cluster/ns) and then deleted, so no need to + // add it to the list + getAsync(joinPath(BASE_POLICIES_PATH, namespace)).thenApply(data -> { + if (data.isPresent()) { + checkNs.completeExceptionally(new IllegalStateException( + "Tenant has active namespace")); + } else { + checkNs.complete(null); + } + return null; + }).exceptionally(ex2 -> { + if (ex2.getCause() instanceof MetadataStoreException.ContentDeserializationException) { + // it's not a valid namespace-node + checkNs.complete(null); + } else { + checkNs.completeExceptionally(ex2); + } + return null; + }); + }); + FutureUtil.waitForAll(activeNamespaceListFuture).thenAccept(r -> { + activeNamespaceFuture.complete(null); + }).exceptionally(ex -> { + activeNamespaceFuture.completeExceptionally(ex.getCause()); + return null; + }); + }); + }).exceptionally(ex -> { + activeNamespaceFuture.completeExceptionally(ex.getCause()); + return null; + }); + return activeNamespaceFuture; + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java index 37125340fbde6..fd89819e9c02b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java @@ -24,8 +24,10 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; public class TopicResources { private static final String MANAGED_LEDGER_PATH = "/managed-ledgers"; @@ -36,17 +38,39 @@ public class TopicResources { this.store = store; } + public CompletableFuture> listPersistentTopicsAsync(NamespaceName ns) { + String path = MANAGED_LEDGER_PATH + "/" + ns + "/persistent"; + + return store.getChildren(path).thenApply(children -> + children.stream().map(c -> TopicName.get(TopicDomain.persistent.toString(), ns, decode(c)).toString()) + .collect(Collectors.toList()) + ); + } + public CompletableFuture> getExistingPartitions(TopicName topic) { - String topicPartitionPath = MANAGED_LEDGER_PATH + "/" + topic.getNamespace() + "/" - + topic.getDomain(); + return getExistingPartitions(topic.getNamespaceObject(), topic.getDomain()); + } + + public CompletableFuture> getExistingPartitions(NamespaceName ns, TopicDomain domain) { + String topicPartitionPath = MANAGED_LEDGER_PATH + "/" + ns + "/" + domain; return store.getChildren(topicPartitionPath).thenApply(topics -> topics.stream() - .map(s -> String.format("%s://%s/%s", - topic.getDomain().value(), topic.getNamespace(), decode(s))) + .map(s -> String.format("%s://%s/%s", domain.value(), ns, decode(s))) .collect(Collectors.toList()) ); } + public CompletableFuture deletePersistentTopicAsync(TopicName topic) { + String path = MANAGED_LEDGER_PATH + "/" + topic.getPersistenceNamingEncoding();; + return store.delete(path, Optional.of(-1l)); + } + + public CompletableFuture createPersistentTopicAsync(TopicName topic) { + String path = MANAGED_LEDGER_PATH + "/" + topic.getPersistenceNamingEncoding();; + return store.put(path, new byte[0], Optional.of(-1l)) + .thenApply(__ -> null); + } + public CompletableFuture persistentTopicExists(TopicName topic) { String path = MANAGED_LEDGER_PATH + "/" + topic.getPersistenceNamingEncoding();; return store.exists(path); @@ -54,16 +78,37 @@ public CompletableFuture persistentTopicExists(TopicName topic) { public CompletableFuture clearNamespacePersistence(NamespaceName ns) { String path = MANAGED_LEDGER_PATH + "/" + ns; - return store.delete(path, Optional.empty()); + return store.exists(path) + .thenCompose(exists -> { + if (exists) { + return store.delete(path, Optional.empty()); + } else { + return CompletableFuture.completedFuture(null); + } + }); } public CompletableFuture clearDomainPersistence(NamespaceName ns) { String path = MANAGED_LEDGER_PATH + "/" + ns + "/persistent"; - return store.delete(path, Optional.empty()); + return store.exists(path) + .thenCompose(exists -> { + if (exists) { + return store.delete(path, Optional.empty()); + } else { + return CompletableFuture.completedFuture(null); + } + }); } - public CompletableFuture clearTennantPersistence(String tenant) { + public CompletableFuture clearTenantPersistence(String tenant) { String path = MANAGED_LEDGER_PATH + "/" + tenant; - return store.delete(path, Optional.empty()); + return store.exists(path) + .thenCompose(exists -> { + if (exists) { + return store.delete(path, Optional.empty()); + } else { + return CompletableFuture.completedFuture(null); + } + }); } } 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 2a3a88e371bcc..43574b8e4c5dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -31,7 +31,8 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.stream.storage.api.cluster.ClusterInitializer; import org.apache.bookkeeper.stream.storage.impl.cluster.ZkClusterInitializer; -import org.apache.pulsar.broker.admin.ZkAdminPaths; +import org.apache.pulsar.broker.resources.NamespaceResources; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.common.conf.InternalConfigurationData; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -339,23 +340,21 @@ static void createNamespaceIfAbsent(MetadataStore configStore, NamespaceName nam static void createPartitionedTopic(MetadataStore configStore, TopicName topicName, int numPartitions) throws InterruptedException, IOException, ExecutionException { - String partitionedTopicPath = ZkAdminPaths.partitionedTopicPath(topicName); - Optional getResult = configStore.get(partitionedTopicPath).get(); - PartitionedTopicMetadata metadata = new PartitionedTopicMetadata(numPartitions); + PulsarResources resources = new PulsarResources(null, configStore); + NamespaceResources.PartitionedTopicResources partitionedTopicResources = + resources.getNamespaceResources().getPartitionedTopicResources(); + + Optional getResult = + partitionedTopicResources.getPartitionedTopicMetadataAsync(topicName).get(); if (!getResult.isPresent()) { - createMetadataNode(configStore, partitionedTopicPath, - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(metadata)); + partitionedTopicResources.createPartitionedTopic(topicName, new PartitionedTopicMetadata(numPartitions)); } else { - byte[] content = getResult.get().getValue(); - PartitionedTopicMetadata existsMeta = - ObjectMapperFactory.getThreadLocal().readValue(content, PartitionedTopicMetadata.class); + PartitionedTopicMetadata existsMeta = getResult.get(); - // Only update z-node if the partitions should be modified + // Only update metadata if the partitions should be modified if (existsMeta.partitions < numPartitions) { - configStore.put( - partitionedTopicPath, - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(metadata), - Optional.of(getResult.get().getStat().getVersion())); + partitionedTopicResources.updatePartitionedTopicAsync(topicName, + __ -> new PartitionedTopicMetadata(numPartitions)).get(); } } } 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 70a0be855e284..5da9772c16199 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 @@ -21,7 +21,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager.DISABLE_RESOURCE_USAGE_TRANSPORT_MANAGER; import static org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl.TRANSACTION_LOG_PREFIX; import com.google.common.annotations.VisibleForTesting; @@ -83,7 +82,6 @@ import org.apache.commons.lang3.builder.ReflectionToStringBuilder; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.ZookeeperSessionExpiredHandlers; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.cache.ConfigurationCacheService; @@ -902,9 +900,9 @@ private void addWebSocketServiceHandler(WebService webService, } private void handleDeleteCluster(Notification notification) { - if (notification.getPath().startsWith(ClusterResources.CLUSTERS_ROOT) + if (ClusterResources.pathRepresentsClusterName(notification.getPath()) && notification.getType() == NotificationType.Deleted) { - final String clusterName = notification.getPath().substring(ClusterResources.CLUSTERS_ROOT.length() + 1); + final String clusterName = ClusterResources.clusterNameFromPath(notification.getPath()); getBrokerService().closeAndRemoveReplicationClient(clusterName); } } @@ -963,9 +961,8 @@ protected void startLeaderElectionService() { protected void acquireSLANamespace() { try { // Namespace not created hence no need to unload it - String nsName = NamespaceService.getSLAMonitorNamespace(getAdvertisedAddress(), config); - if (!this.pulsarResources.getNamespaceResources().exists( - AdminResource.path(POLICIES) + "/" + nsName)) { + NamespaceName nsName = NamespaceService.getSLAMonitorNamespace(getAdvertisedAddress(), config); + if (!this.pulsarResources.getNamespaceResources().namespaceExists(nsName)) { LOG.info("SLA Namespace = {} doesn't exist.", nsName); return; } 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 cf96e923319c2..bf8d1be73647a 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 @@ -18,10 +18,7 @@ */ package org.apache.pulsar.broker.admin; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.common.util.Codec.decode; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Lists; import com.google.errorprone.annotations.CanIgnoreReturnValue; import java.util.ArrayList; import java.util.List; @@ -36,6 +33,7 @@ import javax.ws.rs.container.AsyncResponse; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -66,15 +64,9 @@ import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.apache.pulsar.metadata.api.MetadataStoreException.BadVersionException; -import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +@Slf4j public abstract class AdminResource extends PulsarWebResource { - private static final Logger log = LoggerFactory.getLogger(AdminResource.class); - public static final String POLICIES_READONLY_FLAG_PATH = "/admin/flags/policies-readonly"; - public static final String PARTITIONED_TOPIC_PATH_ZNODE = "partitioned-topics"; - protected BookKeeper bookKeeper() { return pulsar().getBookKeeperClient(); } @@ -128,10 +120,9 @@ public void validatePoliciesReadOnlyAccess() { boolean arePoliciesReadOnly = true; try { - arePoliciesReadOnly = pulsar().getPulsarResources().getNamespaceResources() - .exists(POLICIES_READONLY_FLAG_PATH); + arePoliciesReadOnly = pulsar().getPulsarResources().getNamespaceResources().getPoliciesReadOnly(); } catch (Exception e) { - log.warn("Unable to fetch contents of [{}] from configuration store", POLICIES_READONLY_FLAG_PATH, e); + log.warn("Unable to check if policies are read-only", e); throw new RestException(e); } @@ -162,8 +153,8 @@ private CompletableFuture tryCreatePartitionAsync(final int partition, Com result.completeExceptionally(new IllegalStateException("metadata store not initialized")); return result; } - localStore.get() - .put(ZkAdminPaths.managedLedgerPath(topicName.getPartition(partition)), new byte[0], Optional.of(-1L)) + + getPulsarResources().getTopicResources().createPersistentTopicAsync(topicName.getPartition(partition)) .thenAccept(r -> { if (log.isDebugEnabled()) { log.debug("[{}] Topic partition {} created.", clientAppId(), topicName.getPartition(partition)); @@ -296,8 +287,7 @@ protected void validatePersistentTopicName(String property, String cluster, Stri protected Policies getNamespacePolicies(NamespaceName namespaceName) { try { final String namespace = namespaceName.toString(); - final String policyPath = AdminResource.path(POLICIES, namespace); - Policies policies = namespaceResources().get(policyPath) + Policies policies = namespaceResources().getPolicies(namespaceName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); // fetch bundles from LocalZK-policies BundlesData bundleData = pulsar().getNamespaceService().getNamespaceBundleFactory() @@ -315,10 +305,7 @@ protected Policies getNamespacePolicies(NamespaceName namespaceName) { } protected CompletableFuture getNamespacePoliciesAsync(NamespaceName namespaceName) { - final String namespace = namespaceName.toString(); - final String policyPath = AdminResource.path(POLICIES, namespace); - - return namespaceResources().getAsync(policyPath).thenCompose(policies -> { + return namespaceResources().getPoliciesAsync(namespaceName).thenCompose(policies -> { if (policies.isPresent()) { return pulsar() .getNamespaceService() @@ -352,10 +339,10 @@ protected void mergeNamespaceWithDefaults(Policies policies, String namespace, S } - protected BacklogQuota namespaceBacklogQuota(String namespace, String namespacePath, + protected BacklogQuota namespaceBacklogQuota(NamespaceName namespace, BacklogQuota.BacklogQuotaType backlogQuotaType) { return pulsar().getBrokerService().getBacklogQuotaManager() - .getBacklogQuota(namespace, namespacePath, backlogQuotaType); + .getBacklogQuota(namespace, backlogQuotaType); } protected CompletableFuture> getTopicPoliciesAsyncWithRetry(TopicName topicName) { @@ -526,7 +513,7 @@ protected static PartitionedTopicMetadata fetchPartitionedTopicMetadataCheckAllo protected void validateClusterExists(String cluster) { try { - if (!clusterResources().get(path("clusters", cluster)).isPresent()) { + if (!clusterResources().getCluster(cluster).isPresent()) { throw new RestException(Status.PRECONDITION_FAILED, "Cluster " + cluster + " does not exist."); } } catch (Exception e) { @@ -534,19 +521,21 @@ protected void validateClusterExists(String cluster) { } } - protected Policies getNamespacePolicies(String property, String cluster, String namespace) { + protected Policies getNamespacePolicies(String tenant, String cluster, String namespace) { + NamespaceName ns = NamespaceName.get(tenant, cluster, namespace); + try { - Policies policies = namespaceResources().get(AdminResource.path(POLICIES, property, cluster, namespace)) + Policies policies = namespaceResources().getPolicies(ns) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); // fetch bundles from LocalZK-policies BundlesData bundleData = pulsar().getNamespaceService().getNamespaceBundleFactory() - .getBundles(NamespaceName.get(property, cluster, namespace)).getBundlesData(); + .getBundles(ns).getBundlesData(); policies.bundles = bundleData != null ? bundleData : policies.bundles; return policies; } catch (RestException re) { throw re; } catch (Exception e) { - log.error("[{}] Failed to get namespace policies {}/{}/{}", clientAppId(), property, cluster, namespace, e); + log.error("[{}] Failed to get namespace policies {}", clientAppId(), ns, e); throw new RestException(e); } } @@ -557,7 +546,7 @@ protected boolean isNamespaceReplicated(NamespaceName namespaceName) { protected Set getNamespaceReplicatedClusters(NamespaceName namespaceName) { try { - final Policies policies = namespaceResources().get(ZkAdminPaths.namespacePoliciesPath(namespaceName)) + final Policies policies = namespaceResources().getPolicies(namespaceName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); return policies.replication_clusters; } catch (RestException re) { @@ -569,25 +558,15 @@ protected Set getNamespaceReplicatedClusters(NamespaceName namespaceName } protected List getPartitionedTopicList(TopicDomain topicDomain) { - List partitionedTopics = Lists.newArrayList(); - try { - String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, - namespaceName.toString(), topicDomain.value()); - List topics = namespaceResources().getChildren(partitionedTopicPath); - partitionedTopics = topics.stream() - .map(s -> String.format("%s://%s/%s", topicDomain.value(), namespaceName.toString(), decode(s))) - .collect(Collectors.toList()); - } catch (NotFoundException e) { - // NoNode means there are no partitioned topics in this domain for this namespace + return namespaceResources().getPartitionedTopicResources() + .listPartitionedTopicsAsync(namespaceName, topicDomain) + .join(); } catch (Exception e) { log.error("[{}] Failed to get partitioned topic list for namespace {}", clientAppId(), namespaceName.toString(), e); throw new RestException(e); } - - partitionedTopics.sort(null); - return partitionedTopics; } protected List getTopicPartitionList(TopicDomain topicDomain) { @@ -687,16 +666,6 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n return null; }); - if (!createLocalTopicOnly && topicName.isGlobal() && isNamespaceReplicated(namespaceName)) { - getNamespaceReplicatedClusters(namespaceName) - .stream() - .filter(cluster -> !cluster.equals(pulsar().getConfiguration().getClusterName())) - .forEach(cluster -> createFutureList.add( - ((TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster).topics()) - .createPartitionedTopicAsync( - topicName.getPartitionedTopicName(), numPartitions, true))); - } - FutureUtil.waitForAll(createFutureList).whenComplete((ignored, ex) -> { if (ex != null) { log.error("[{}] Failed to create partitions for topic {}", clientAppId(), topicName, ex.getCause()); @@ -707,6 +676,17 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n } return; } + + if (!createLocalTopicOnly && topicName.isGlobal() && isNamespaceReplicated(namespaceName)) { + getNamespaceReplicatedClusters(namespaceName) + .stream() + .filter(cluster -> !cluster.equals(pulsar().getConfiguration().getClusterName())) + .forEach(cluster -> createFutureList.add( + ((TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster).topics()) + .createPartitionedTopicAsync( + topicName.getPartitionedTopicName(), numPartitions, true))); + } + log.info("[{}] Successfully created partitions for topic {} in cluster {}", clientAppId(), topicName, pulsar().getConfiguration().getClusterName()); asyncResponse.resume(Response.noContent().build()); @@ -741,10 +721,9 @@ private CompletableFuture provisionPartitionedTopicPath(AsyncResponse asyn int numPartitions, boolean createLocalTopicOnly) { CompletableFuture future = new CompletableFuture<>(); - String partitionedTopicPath = ZkAdminPaths.partitionedTopicPath(topicName); namespaceResources() .getPartitionedTopicResources() - .createAsync(partitionedTopicPath, new PartitionedTopicMetadata(numPartitions)) + .createPartitionedTopicAsync(topicName, new PartitionedTopicMetadata(numPartitions)) .whenComplete((ignored, ex) -> { if (ex != null) { if (ex instanceof AlreadyExistsException) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ZkAdminPaths.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ZkAdminPaths.java deleted file mode 100644 index 95954f610d25e..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ZkAdminPaths.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.admin; - -import org.apache.pulsar.common.naming.NamespaceName; -import org.apache.pulsar.common.naming.TopicName; - -public class ZkAdminPaths { - - public static final String POLICIES = "policies"; - public static final String PARTITIONED_TOPIC_PATH_ZNODE = "partitioned-topics"; - - public static String partitionedTopicPath(TopicName name) { - return adminPath(PARTITIONED_TOPIC_PATH_ZNODE, - name.getNamespace(), name.getDomain().value(), name.getEncodedLocalName()); - } - - public static String managedLedgerPath(TopicName name) { - return "/managed-ledgers/" + name.getPersistenceNamingEncoding(); - } - - public static String namespacePoliciesPath(NamespaceName name) { - return adminPath(POLICIES, name.toString()); - } - - private static String adminPath(String... parts) { - return "/admin/" + String.join("/", parts); - } - - private ZkAdminPaths() {} -} 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 eef90813e01f4..17f497c2e8fe3 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 @@ -18,15 +18,12 @@ */ package org.apache.pulsar.broker.admin.impl; -import static org.apache.pulsar.broker.service.BrokerService.BROKER_SERVICE_CONFIGURATION_PATH; import com.google.common.collect.Maps; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; import java.time.Duration; -import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -47,7 +44,6 @@ import org.apache.pulsar.broker.PulsarService.State; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.LeaderBroker; -import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.web.PulsarWebResource; @@ -59,6 +55,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.conf.InternalConfigurationData; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.policies.data.BrokerInfo; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; @@ -91,8 +88,7 @@ public Set getActiveBrokers(@PathParam("cluster") String cluster) throws validateClusterOwnership(cluster); try { - // Add Native brokers - return new HashSet<>(dynamicConfigurationResources().getChildren(LoadManager.LOADBALANCE_BROKERS_ROOT)); + return pulsar().getLoadManager().get().getAvailableBrokers(); } catch (Exception e) { LOG.error("[{}] Failed to get active broker list: cluster={}", clientAppId(), cluster, e); throw new RestException(e); @@ -185,8 +181,7 @@ public void deleteDynamicConfiguration(@PathParam("configName") String configNam public Map getAllDynamicConfigurations() throws Exception { validateSuperUserAccess(); try { - return dynamicConfigurationResources().get(BROKER_SERVICE_CONFIGURATION_PATH) - .orElseGet(() -> Collections.emptyMap()); + return dynamicConfigurationResources().getDynamicConfiguration(); } catch (RestException e) { LOG.error("[{}] couldn't find any configuration in zk {}", clientAppId(), e.getMessage(), e); throw e; @@ -230,7 +225,7 @@ private synchronized void persistDynamicConfiguration(String configName, String throw new RestException(Status.PRECONDITION_FAILED, " Invalid dynamic-config value"); } if (BrokerService.isDynamicConfiguration(configName)) { - dynamicConfigurationResources().setWithCreate(BROKER_SERVICE_CONFIGURATION_PATH, (old) -> { + dynamicConfigurationResources().setDynamicConfigurationWithCreate(old -> { Map configurationMap = old.isPresent() ? old.get() : Maps.newHashMap(); configurationMap.put(configName, configValue); return configurationMap; @@ -310,9 +305,7 @@ public void healthcheck(@Suspended AsyncResponse asyncResponse, PulsarClient client; try { validateSuperUserAccess(); - String heartbeatNamespace; - - heartbeatNamespace = (topicVersion == TopicVersion.V2) + NamespaceName heartbeatNamespace = (topicVersion == TopicVersion.V2) ? NamespaceService.getHeartbeatNamespaceV2( pulsar().getAdvertisedAddress(), @@ -421,7 +414,7 @@ private void healthcheckReadLoop(CompletableFuture> readerFuture, private synchronized void deleteDynamicConfigurationOnZk(String configName) { try { if (BrokerService.isDynamicConfiguration(configName)) { - dynamicConfigurationResources().set(BROKER_SERVICE_CONFIGURATION_PATH, (old) -> { + dynamicConfigurationResources().setDynamicConfiguration(old -> { if (old != null) { old.remove(configName); } 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 6ccbc8c330a11..d8e76d0c3f6c4 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 @@ -18,8 +18,6 @@ */ package org.apache.pulsar.broker.admin.impl; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.broker.namespace.NamespaceService.NAMESPACE_ISOLATION_POLICIES; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.swagger.annotations.ApiOperation; @@ -49,8 +47,6 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; -import org.apache.pulsar.broker.admin.AdminResource; -import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.resources.ClusterResources.FailureDomainResources; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; @@ -118,7 +114,7 @@ public ClusterData getCluster( validateSuperUserAccess(); try { - return clusterResources().get(path("clusters", cluster)) + return clusterResources().getCluster(cluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster does not exist")); } catch (Exception e) { log.error("[{}] Failed to get cluster {}", clientAppId(), cluster, e); @@ -170,11 +166,11 @@ public void createCluster( try { NamedEntity.checkName(cluster); - if (clusterResources().get(path("clusters", cluster)).isPresent()) { + if (clusterResources().getCluster(cluster).isPresent()) { log.warn("[{}] Failed to create already existing cluster {}", clientAppId(), cluster); throw new RestException(Status.CONFLICT, "Cluster already exists"); } - clusterResources().create(path("clusters", cluster), clusterData); + clusterResources().createCluster(cluster, clusterData); log.info("[{}] Created cluster {}", clientAppId(), cluster); } catch (IllegalArgumentException e) { log.warn("[{}] Failed to create cluster with invalid name {}", clientAppId(), cluster, e); @@ -222,7 +218,7 @@ public void updateCluster( validatePoliciesReadOnlyAccess(); try { - clusterResources().set(path("clusters", cluster), old -> clusterData); + clusterResources().updateCluster(cluster, old -> clusterData); log.info("[{}] Updated cluster {}", clientAppId(), cluster); } catch (NotFoundException e) { log.warn("[{}] Failed to update cluster {}: Does not exist", clientAppId(), cluster); @@ -278,7 +274,7 @@ public void setPeerClusterNames( throw new RestException(Status.PRECONDITION_FAILED, cluster + " itself can't be part of peer-list"); } - clusterResources().get(path("clusters", peerCluster)) + clusterResources().getCluster(peerCluster) .orElseThrow(() -> new RestException(Status.PRECONDITION_FAILED, "Peer cluster " + peerCluster + " does not exist")); } catch (RestException e) { @@ -294,7 +290,7 @@ public void setPeerClusterNames( } try { - clusterResources().set(path("clusters", cluster), old -> + clusterResources().updateCluster(cluster, old -> old.clone() .peerClusterNames(peerClusterNames) .build() @@ -331,7 +327,7 @@ public Set getPeerCluster( ) { validateSuperUserAccess(); try { - ClusterData clusterData = clusterResources().get(path("clusters", cluster)) + ClusterData clusterData = clusterResources().getCluster(cluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster does not exist")); return clusterData.getPeerClusterNames(); } catch (Exception e) { @@ -363,29 +359,19 @@ public void deleteCluster( validateSuperUserAccess(); validatePoliciesReadOnlyAccess(); - // Check that the cluster is not used by any property (eg: no namespaces provisioned there) + // Check that the cluster is not used by any tenant (eg: no namespaces provisioned there) boolean isClusterUsed = false; try { - for (String property : tenantResources().getChildren(path(POLICIES))) { - if (!clusterResources().exists(path(POLICIES, property, cluster))) { - continue; - } - - if (!clusterResources().getChildren(path(POLICIES, property, cluster)).isEmpty()) { - // We found a property that has at least a namespace in this cluster - isClusterUsed = true; - break; - } - } + isClusterUsed = pulsar().getPulsarResources().getClusterResources().isClusterUsed(cluster); // check the namespaceIsolationPolicies associated with the cluster - String path = path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); - Optional nsIsolationPolicies = namespaceIsolationPolicies().getPolicies(path); + Optional nsIsolationPolicies = + namespaceIsolationPolicies().getIsolationDataPolicies(cluster); // Need to delete the isolation policies if present if (nsIsolationPolicies.isPresent()) { if (nsIsolationPolicies.get().getPolicies().isEmpty()) { - namespaceIsolationPolicies().delete(path); + namespaceIsolationPolicies().deleteIsolationData(cluster); } else { isClusterUsed = true; } @@ -401,9 +387,8 @@ public void deleteCluster( } try { - String clusterPath = path("clusters", cluster); - deleteFailureDomain(clusterPath); - clusterResources().delete(clusterPath); + clusterResources().getFailureDomainResources().deleteFailureDomains(cluster); + clusterResources().deleteCluster(cluster); log.info("[{}] Deleted cluster {}", clientAppId(), cluster); } catch (NotFoundException e) { log.warn("[{}] Failed to delete cluster {} - Does not exist", clientAppId(), cluster); @@ -414,23 +399,6 @@ public void deleteCluster( } } - private void deleteFailureDomain(String clusterPath) { - try { - String failureDomain = joinPath(clusterPath, ConfigurationCacheService.FAILURE_DOMAIN); - if (!clusterResources().exists(failureDomain)) { - return; - } - for (String domain : clusterResources().getChildren(failureDomain)) { - String domainPath = joinPath(failureDomain, domain); - clusterResources().delete(domainPath); - } - clusterResources().delete(failureDomain); - } catch (Exception e) { - log.warn("Failed to delete failure-domain under cluster {}", clusterPath); - throw new RestException(e); - } - } - @GET @Path("/{cluster}/namespaceIsolationPolicies") @ApiOperation( @@ -452,13 +420,13 @@ private void deleteFailureDomain(String clusterPath) { @PathParam("cluster") String cluster ) throws Exception { validateSuperUserAccess(); - if (!clusterResources().exists(path("clusters", cluster))) { + if (!clusterResources().clusterExists(cluster)) { throw new RestException(Status.NOT_FOUND, "Cluster " + cluster + " does not exist."); } try { NamespaceIsolationPolicies nsIsolationPolicies = namespaceIsolationPolicies() - .getPolicies(path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES)) + .getIsolationDataPolicies(cluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "NamespaceIsolationPolicies for cluster " + cluster + " does not exist")); // construct the response to Namespace isolation data map @@ -499,7 +467,7 @@ public NamespaceIsolationData getNamespaceIsolationPolicy( try { NamespaceIsolationPolicies nsIsolationPolicies = namespaceIsolationPolicies() - .getPolicies(path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES)) + .getIsolationDataPolicies(cluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "NamespaceIsolationPolicies for cluster " + cluster + " does not exist")); // construct the response to Namespace isolation data map @@ -542,7 +510,6 @@ public List getBrokersWithNamespaceIsolationPolicy validateClusterExists(cluster); Set availableBrokers; - final String nsIsolationPoliciesPath = AdminResource.path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); Map nsPolicies; try { availableBrokers = pulsar().getLoadManager().get().getAvailableBrokers(); @@ -552,7 +519,7 @@ public List getBrokersWithNamespaceIsolationPolicy } try { Optional nsPoliciesResult = namespaceIsolationPolicies() - .getPolicies(nsIsolationPoliciesPath); + .getIsolationDataPolicies(cluster); if (!nsPoliciesResult.isPresent()) { throw new RestException(Status.NOT_FOUND, "namespace-isolation policies not found for " + cluster); } @@ -610,11 +577,10 @@ public BrokerNamespaceIsolationData getBrokerWithNamespaceIsolationPolicy( validateSuperUserAccess(); validateClusterExists(cluster); - final String nsIsolationPoliciesPath = AdminResource.path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); Map nsPolicies; try { Optional nsPoliciesResult = namespaceIsolationPolicies() - .getPolicies(nsIsolationPoliciesPath); + .getIsolationDataPolicies(cluster); if (!nsPoliciesResult.isPresent()) { throw new RestException(Status.NOT_FOUND, "namespace-isolation policies not found for " + cluster); } @@ -682,11 +648,10 @@ public void setNamespaceIsolationPolicy( policyData.validate(); jsonInput = ObjectMapperFactory.create().writeValueAsString(policyData); - String nsIsolationPolicyPath = path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); NamespaceIsolationPolicies nsIsolationPolicies = namespaceIsolationPolicies() - .getPolicies(nsIsolationPolicyPath).orElseGet(() -> { + .getIsolationDataPolicies(cluster).orElseGet(() -> { try { - namespaceIsolationPolicies().setWithCreate(nsIsolationPolicyPath, + namespaceIsolationPolicies().setIsolationDataWithCreate(cluster, (p) -> Collections.emptyMap()); return new NamespaceIsolationPolicies(); } catch (Exception e) { @@ -695,7 +660,7 @@ public void setNamespaceIsolationPolicy( }); nsIsolationPolicies.setPolicy(policyName, policyData); - namespaceIsolationPolicies().set(nsIsolationPolicyPath, old -> nsIsolationPolicies.getPolicies()); + namespaceIsolationPolicies().setIsolationData(cluster, old -> nsIsolationPolicies.getPolicies()); // whether or not make the isolation update on time. if (pulsar().getConfiguration().isEnableNamespaceIsolationUpdateOnTime()) { @@ -833,11 +798,10 @@ public void deleteNamespaceIsolationPolicy( try { - String nsIsolationPolicyPath = path("clusters", cluster, NAMESPACE_ISOLATION_POLICIES); NamespaceIsolationPolicies nsIsolationPolicies = namespaceIsolationPolicies() - .getPolicies(nsIsolationPolicyPath).orElseGet(() -> { + .getIsolationDataPolicies(cluster).orElseGet(() -> { try { - namespaceIsolationPolicies().setWithCreate(nsIsolationPolicyPath, + namespaceIsolationPolicies().setIsolationDataWithCreate(cluster, (p) -> Collections.emptyMap()); return new NamespaceIsolationPolicies(); } catch (Exception e) { @@ -846,7 +810,7 @@ public void deleteNamespaceIsolationPolicy( }); nsIsolationPolicies.deletePolicy(policyName); - namespaceIsolationPolicies().set(nsIsolationPolicyPath, old -> nsIsolationPolicies.getPolicies()); + namespaceIsolationPolicies().setIsolationData(cluster, old -> nsIsolationPolicies.getPolicies()); } catch (NotFoundException nne) { log.warn("[{}] Failed to update brokers/{}/namespaceIsolationPolicies: Does not exist", clientAppId(), cluster); @@ -894,9 +858,8 @@ public void setFailureDomain( validateBrokerExistsInOtherDomain(cluster, domainName, domain); try { - String domainPath = joinPath(pulsar().getConfigurationCache().CLUSTER_FAILURE_DOMAIN_ROOT, domainName); - FailureDomainResources failureDomainListCache = clusterResources().getFailureDomainResources(); - failureDomainListCache.setWithCreate(domainPath, old -> domain); + clusterResources().getFailureDomainResources() + .setFailureDomainWithCreate(cluster, domainName, old -> domain); } catch (NotFoundException nne) { log.warn("[{}] Failed to update domain {}. clusters {} Does not exist", clientAppId(), cluster, domainName); @@ -931,12 +894,10 @@ public Map getFailureDomains( Map domains = Maps.newHashMap(); try { - final String failureDomainRootPath = pulsar().getConfigurationCache().CLUSTER_FAILURE_DOMAIN_ROOT; - FailureDomainResources failureDomainListCache = clusterResources().getFailureDomainResources(); - for (String domainName : failureDomainListCache.getChildren(failureDomainRootPath)) { + FailureDomainResources fdr = clusterResources().getFailureDomainResources(); + for (String domainName : fdr.listFailureDomains(cluster)) { try { - Optional domain = failureDomainListCache - .get(joinPath(failureDomainRootPath, domainName)); + Optional domain = fdr.getFailureDomain(cluster, domainName); domain.ifPresent(failureDomain -> domains.put(domainName, failureDomain)); } catch (Exception e) { log.warn("Failed to get domain {}", domainName, e); @@ -982,7 +943,7 @@ public FailureDomainImpl getDomain( try { final String failureDomainRootPath = pulsar().getConfigurationCache().CLUSTER_FAILURE_DOMAIN_ROOT; - return clusterResources().getFailureDomainResources().get(joinPath(failureDomainRootPath, domainName)) + return clusterResources().getFailureDomainResources().getFailureDomain(cluster, domainName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Domain " + domainName + " for cluster " + cluster + " does not exist")); } catch (RestException re) { @@ -1021,9 +982,7 @@ public void deleteFailureDomain( validateClusterExists(cluster); try { - final String domainPath = joinPath(pulsar().getConfigurationCache().CLUSTER_FAILURE_DOMAIN_ROOT, - domainName); - clusterResources().getFailureDomainResources().delete(domainPath); + clusterResources().getFailureDomainResources().deleteFailureDomain(cluster, domainName); } catch (NotFoundException nne) { log.warn("[{}] Domain {} does not exist in {}", clientAppId(), domainName, cluster); throw new RestException(Status.NOT_FOUND, @@ -1038,16 +997,14 @@ private void validateBrokerExistsInOtherDomain(final String cluster, final Strin final FailureDomainImpl inputDomain) { if (inputDomain != null && inputDomain.brokers != null) { try { - final String failureDomainRootPath = pulsar().getConfigurationCache().CLUSTER_FAILURE_DOMAIN_ROOT; for (String domainName : clusterResources().getFailureDomainResources() - .getChildren(failureDomainRootPath)) { + .listFailureDomains(cluster)) { if (inputDomainName.equals(domainName)) { continue; } try { Optional domain = - clusterResources().getFailureDomainResources() - .get(joinPath(failureDomainRootPath, domainName)); + clusterResources().getFailureDomainResources().getFailureDomain(cluster, domainName); if (domain.isPresent() && domain.get().brokers != null) { List duplicateBrokers = domain.get().brokers.stream().parallel() .filter(inputDomain.brokers::contains).collect(Collectors.toList()); 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 738d22eb4bc11..b6a1a547dce37 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 @@ -20,11 +20,6 @@ package org.apache.pulsar.broker.admin.impl; import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.RESOURCEGROUPS; -import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; -import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.MANAGED_LEDGER_ROOT; -import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.OWNER_INFO_ROOT; import static org.apache.pulsar.common.policies.data.PoliciesUtil.defaultBundle; import static org.apache.pulsar.common.policies.data.PoliciesUtil.getBundles; import com.google.common.collect.Lists; @@ -107,7 +102,6 @@ import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.apache.pulsar.metadata.api.MetadataStoreException.BadVersionException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; -import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -125,10 +119,11 @@ protected List internalGetTenantNamespaces(String tenant) { validateTenantOperation(tenant, TenantOperation.LIST_NAMESPACES); try { - return getListOfNamespaces(tenant); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get namespace list for tenant: {} - Does not exist", clientAppId(), tenant); - throw new RestException(Status.NOT_FOUND, "Property does not exist"); + if (!tenantResources().tenantExists(tenant)) { + throw new RestException(Status.NOT_FOUND, "Tenant not found"); + } + + return tenantResources().getListOfNamespaces(tenant); } catch (Exception e) { log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e); throw new RestException(e); @@ -144,13 +139,13 @@ protected void internalCreateNamespace(Policies policies) { int maxNamespacesPerTenant = pulsar().getConfiguration().getMaxNamespacesPerTenant(); // no distributed locks are added here.In a concurrent scenario, the threshold will be exceeded. if (maxNamespacesPerTenant > 0) { - List namespaces = getListOfNamespaces(namespaceName.getTenant()); + List namespaces = tenantResources().getListOfNamespaces(namespaceName.getTenant()); if (namespaces != null && namespaces.size() > maxNamespacesPerTenant) { throw new RestException(Status.PRECONDITION_FAILED, "Exceed the maximum number of namespace in tenant :" + namespaceName.getTenant()); } } - namespaceResources().create(path(POLICIES, namespaceName.toString()), policies); + namespaceResources().createPolicies(namespaceName, policies); log.info("[{}] Created namespace {}", clientAppId(), namespaceName); } catch (AlreadyExistsException e) { log.warn("[{}] Failed to create namespace {} - already exists", clientAppId(), namespaceName); @@ -184,7 +179,7 @@ protected void internalDeleteNamespace(AsyncResponse asyncResponse, boolean auth // ensure the local cluster is the only cluster for the global namespace configuration try { - policies = namespaceResources().get(path(POLICIES, namespaceName.toString())).orElseThrow( + policies = namespaceResources().getPolicies(namespaceName).orElseThrow( () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist.")); if (namespaceName.isGlobal()) { if (policies.replication_clusters.size() > 1) { @@ -196,8 +191,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); - ClusterData replClusterData = clusterResources().get( - AdminResource.path("clusters", replCluster)) + ClusterData replClusterData = clusterResources().getCluster(replCluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); URL replClusterUrl; @@ -259,7 +253,7 @@ protected void internalDeleteNamespace(AsyncResponse asyncResponse, boolean auth // set the policies to deleted so that somebody else cannot acquire this namespace try { - namespaceResources().set(path(POLICIES, namespaceName.toString()), (old) -> { + namespaceResources().setPolicies(namespaceName, old -> { old.deleted = true; return old; }); @@ -308,45 +302,36 @@ protected void internalDeleteNamespace(AsyncResponse asyncResponse, boolean auth } try { - final String globalPartitionedPath = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString()); - final String managedLedgerPath = joinPath(MANAGED_LEDGER_ROOT, namespaceName.toString()); - // check whether partitioned topics znode exist - if (namespaceResources().exists(globalPartitionedPath)) { - deleteRecursive(namespaceResources(), globalPartitionedPath); - } + namespaceResources().getPartitionedTopicResources().clearPartitionedTopicMetadata(namespaceName); try { - if (namespaceResources().exists(managedLedgerPath)) { - pulsar().getPulsarResources().getTopicResources() - .clearDomainPersistence(namespaceName).get(); - pulsar().getPulsarResources().getTopicResources() - .clearNamespacePersistence(namespaceName).get(); - } + pulsar().getPulsarResources().getTopicResources() + .clearDomainPersistence(namespaceName).get(); + pulsar().getPulsarResources().getTopicResources() + .clearNamespacePersistence(namespaceName).get(); } catch (ExecutionException | InterruptedException e) { // warn level log here since this failure has no side effect besides left a un-used metadata // and also will not affect the re-creation of namespace log.warn("[{}] Failed to remove managed-ledger for {}", clientAppId(), namespaceName, e); } + // we have successfully removed all the ownership for the namespace, the policies znode can be deleted // now - final String globalZkPolicyPath = path(POLICIES, namespaceName.toString()); - final String localZkPolicyPath = joinPath(LOCAL_POLICIES_ROOT, namespaceName.toString()); - final String namespacePath = joinPath(OWNER_INFO_ROOT, namespaceName.toString()); - namespaceResources().delete(globalZkPolicyPath); + namespaceResources().deletePolicies(namespaceName); try { - namespaceResources().delete(namespacePath); + namespaceResources().deletePolicies(namespaceName); } catch (NotFoundException e) { - // If the z-node with the modified information is not there anymore, we're already good + // If the node with the modified information is not there anymore, we're already good } try { - getLocalPolicies().delete(localZkPolicyPath); + getLocalPolicies().deleteLocalPolicies(namespaceName); } catch (NotFoundException nne) { // If the z-node with the modified information is not there anymore, we're already good } } catch (Exception e) { - log.error("[{}] Failed to remove owned namespace {} from ZK", clientAppId(), namespaceName, e); + log.error("[{}] Failed to remove owned namespace {} from metadata", clientAppId(), namespaceName, e); asyncResponse.resume(new RestException(e)); return null; } @@ -376,7 +361,7 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo // ensure the local cluster is the only cluster for the global namespace configuration try { - policies = namespaceResources().get(path(POLICIES, namespaceName.toString())).orElseThrow( + policies = namespaceResources().getPolicies(namespaceName).orElseThrow( () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist.")); if (namespaceName.isGlobal()) { if (policies.replication_clusters.size() > 1) { @@ -389,7 +374,7 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); ClusterData replClusterData = - clusterResources().get(AdminResource.path("clusters", replCluster)) + clusterResources().getCluster(replCluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); URL replClusterUrl; @@ -429,7 +414,7 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo // set the policies to deleted so that somebody else cannot acquire this namespace try { - namespaceResources().set(path(POLICIES, namespaceName.toString()), (old) -> { + namespaceResources().setPolicies(namespaceName, old -> { old.deleted = true; return old; }); @@ -511,11 +496,8 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo try { // remove partitioned topics znode - final String globalPartitionedPath = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString()); - // check whether partitioned topics znode exist - if (namespaceResources().exists(globalPartitionedPath)) { - deleteRecursive(namespaceResources(), globalPartitionedPath); - } + pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .clearPartitionedTopicMetadata(namespaceName); try { pulsar().getPulsarResources().getTopicResources().clearDomainPersistence(namespaceName).get(); @@ -528,19 +510,10 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo // we have successfully removed all the ownership for the namespace, the policies znode can be deleted // now - final String globalZkPolicyPath = path(POLICIES, namespaceName.toString()); - final String localZkPolicyPath = joinPath(LOCAL_POLICIES_ROOT, namespaceName.toString()); - final String namespacePath = joinPath(OWNER_INFO_ROOT, namespaceName.toString()); - namespaceResources().delete(globalZkPolicyPath); + namespaceResources().deletePolicies(namespaceName); try { - namespaceResources().delete(namespacePath); - } catch (NotFoundException e) { - // If the z-node with the modified information is not there anymore, we're already good - } - - try { - getLocalPolicies().delete(localZkPolicyPath); + getLocalPolicies().deleteLocalPolicies(namespaceName); } catch (NotFoundException nne) { // If the z-node with the modified information is not there anymore, we're already good } @@ -587,7 +560,7 @@ protected void internalDeleteNamespaceBundle(String bundleRange, boolean authori // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); ClusterData replClusterData = - clusterResources().get(AdminResource.path("clusters", replCluster)) + clusterResources().getCluster(replCluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); URL replClusterUrl; @@ -662,7 +635,7 @@ protected void internalDeleteNamespaceBundleForcefully(String bundleRange, boole // the only replication cluster is other cluster, redirect String replCluster = Lists.newArrayList(policies.replication_clusters).get(0); ClusterData replClusterData = - clusterResources().get(AdminResource.path("clusters", replCluster)) + clusterResources().getCluster(replCluster) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Cluster " + replCluster + " does not exist")); URL replClusterUrl; @@ -773,7 +746,7 @@ protected void internalRevokePermissionsOnNamespace(String role) { validateNamespaceOperation(namespaceName, NamespaceOperation.REVOKE_PERMISSION); validatePoliciesReadOnlyAccess(); checkNotNull(role, "Role should not be null"); - updatePolicies(path(POLICIES, namespaceName.toString()), policies ->{ + updatePolicies(namespaceName, policies ->{ policies.auth_policies.getNamespaceAuthentication().remove(role); return policies; }); @@ -829,7 +802,7 @@ protected void internalSetNamespaceReplicationClusters(List clusterIds) validatePeerClusterConflict(clusterId, replicationClusterSet); validateClusterForTenant(namespaceName.getTenant(), clusterId); } - updatePolicies(path(POLICIES, namespaceName.toString()), policies ->{ + updatePolicies(namespaceName, policies ->{ policies.replication_clusters = replicationClusterSet; return policies; }); @@ -841,7 +814,7 @@ protected void internalSetNamespaceMessageTTL(Integer messageTTL) { if (messageTTL != null && messageTTL < 0) { throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL"); } - updatePolicies(path(POLICIES, namespaceName.toString()), (policies)->{ + updatePolicies(namespaceName, policies -> { policies.message_ttl_in_seconds = messageTTL; return policies; }); @@ -854,7 +827,7 @@ protected void internalSetSubscriptionExpirationTime(Integer expirationTime) { if (expirationTime != null && expirationTime < 0) { throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for subscription expiration time"); } - updatePolicies(path(POLICIES, namespaceName.toString()), (policies) -> { + updatePolicies(namespaceName, policies -> { policies.subscription_expiration_time_minutes = expirationTime; return policies; }); @@ -876,7 +849,7 @@ protected void internalSetAutoTopicCreation(AsyncResponse asyncResponse, } } // Force to read the data s.t. the watch to the cache content is setup. - namespaceResources().setAsync(path(POLICIES, namespaceName.toString()), (policies) -> { + namespaceResources().setPoliciesAsync(namespaceName, policies -> { policies.autoTopicCreationOverride = autoTopicCreationOverride; return policies; }).thenApply(r -> { @@ -908,7 +881,7 @@ protected void internalSetAutoSubscriptionCreation( validatePoliciesReadOnlyAccess(); // Force to read the data s.t. the watch to the cache content is setup. - namespaceResources().setAsync(path(POLICIES, namespaceName.toString()), (policies) -> { + namespaceResources().setPoliciesAsync(namespaceName, policies -> { policies.autoSubscriptionCreationOverride = autoSubscriptionCreationOverride; return policies; }).thenApply(r -> { @@ -939,7 +912,7 @@ protected void internalRemoveAutoSubscriptionCreation(AsyncResponse asyncRespons protected void internalModifyDeduplication(Boolean enableDeduplication) { validateNamespacePolicyOperation(namespaceName, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); - updatePolicies(path(POLICIES, namespaceName.toString()), policies ->{ + updatePolicies(namespaceName, policies -> { policies.deduplicationEnabled = enableDeduplication; return policies; }); @@ -1005,9 +978,8 @@ protected void internalSetBookieAffinityGroup(BookieAffinityGroupData bookieAffi validateClusterForTenant(namespaceName.getTenant(), namespaceName.getCluster()); } - String path = joinPath(LOCAL_POLICIES_ROOT, this.namespaceName.toString()); try { - getLocalPolicies().setWithCreate(path, (oldPolicies) -> { + getLocalPolicies().setLocalPoliciesWithCreate(namespaceName, oldPolicies -> { LocalPolicies localPolicies = oldPolicies.map( policies -> new LocalPolicies(policies.bundles, bookieAffinityGroup, @@ -1044,9 +1016,8 @@ protected BookieAffinityGroupData internalGetBookieAffinityGroup() { validateClusterOwnership(namespaceName.getCluster()); validateClusterForTenant(namespaceName.getTenant(), namespaceName.getCluster()); } - String path = joinPath(LOCAL_POLICIES_ROOT, this.namespaceName.toString()); try { - final BookieAffinityGroupData bookkeeperAffinityGroup = getLocalPolicies().get(path) + final BookieAffinityGroupData bookkeeperAffinityGroup = getLocalPolicies().getLocalPolicies(namespaceName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace local-policies does not exist")).bookieAffinityGroup; if (bookkeeperAffinityGroup == null) { @@ -1060,8 +1031,8 @@ protected BookieAffinityGroupData internalGetBookieAffinityGroup() { } catch (RestException re) { throw re; } catch (Exception e) { - log.error("[{}] Failed to get local-policy configuration for namespace {} at path {}", clientAppId(), - namespaceName, path, e); + log.error("[{}] Failed to get local-policy configuration for namespace {}", clientAppId(), + namespaceName, e); throw new RestException(e); } } @@ -1210,8 +1181,7 @@ private NamespaceBundleSplitAlgorithm getNamespaceBundleSplitAlgorithmByName(Str protected void internalSetPublishRate(PublishRate maxPublishMessageRate) { validateSuperUserAccess(); log.info("[{}] Set namespace publish-rate {}/{}", clientAppId(), namespaceName, maxPublishMessageRate); - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.publishMaxMessageRate.put(pulsar().getConfiguration().getClusterName(), maxPublishMessageRate); return policies; }); @@ -1223,8 +1193,7 @@ protected void internalRemovePublishRate() { validateSuperUserAccess(); log.info("[{}] Remove namespace publish-rate {}/{}", clientAppId(), namespaceName, topicName); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { if (policies.publishMaxMessageRate != null) { policies.publishMaxMessageRate.remove(pulsar().getConfiguration().getClusterName()); } @@ -1258,8 +1227,7 @@ protected void internalSetTopicDispatchRate(DispatchRateImpl dispatchRate) { log.info("[{}] Set namespace dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies)->{ + updatePolicies(namespaceName, policies -> { policies.topicDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); policies.clusterDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); return policies; @@ -1276,8 +1244,7 @@ protected void internalSetTopicDispatchRate(DispatchRateImpl dispatchRate) { protected void internalDeleteTopicDispatchRate() { validateSuperUserAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.topicDispatchRate.remove(pulsar().getConfiguration().getClusterName()); policies.clusterDispatchRate.remove(pulsar().getConfiguration().getClusterName()); return policies; @@ -1304,8 +1271,7 @@ protected void internalSetSubscriptionDispatchRate(DispatchRateImpl dispatchRate log.info("[{}] Set namespace subscription dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, (policies) -> { policies.subscriptionDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); return policies; }); @@ -1322,8 +1288,7 @@ protected void internalDeleteSubscriptionDispatchRate() { validateSuperUserAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.subscriptionDispatchRate.remove(pulsar().getConfiguration().getClusterName()); return policies; }); @@ -1347,8 +1312,7 @@ protected void internalSetSubscribeRate(SubscribeRate subscribeRate) { validateSuperUserAccess(); log.info("[{}] Set namespace subscribe-rate {}/{}", clientAppId(), namespaceName, subscribeRate); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.clusterSubscribeRate.put(pulsar().getConfiguration().getClusterName(), subscribeRate); return policies; }); @@ -1364,8 +1328,7 @@ protected void internalSetSubscribeRate(SubscribeRate subscribeRate) { protected void internalDeleteSubscribeRate() { validateSuperUserAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.clusterSubscribeRate.remove(pulsar().getConfiguration().getClusterName()); return policies; }); @@ -1387,8 +1350,7 @@ protected SubscribeRate internalGetSubscribeRate() { protected void internalRemoveReplicatorDispatchRate() { validateSuperUserAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.replicatorDispatchRate.remove(pulsar().getConfiguration().getClusterName()); return policies; }); @@ -1405,8 +1367,7 @@ protected void internalSetReplicatorDispatchRate(DispatchRateImpl dispatchRate) validateSuperUserAccess(); log.info("[{}] Set namespace replicator dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.replicatorDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); return policies; }); @@ -1432,8 +1393,7 @@ protected void internalSetBacklogQuota(BacklogQuotaType backlogQuotaType, Backlo final BacklogQuotaType quotaType = backlogQuotaType != null ? backlogQuotaType : BacklogQuotaType.destination_storage; try { - final String path = path(POLICIES, namespaceName.toString()); - Policies policies = namespaceResources().get(path) + Policies policies = namespaceResources().getPolicies(namespaceName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace policies does not exist")); RetentionPolicies r = policies.retention_policies; if (r != null) { @@ -1450,7 +1410,7 @@ protected void internalSetBacklogQuota(BacklogQuotaType backlogQuotaType, Backlo } } policies.backlog_quota_map.put(quotaType, backlogQuota); - namespaceResources().set(path, p -> policies); + namespaceResources().setPolicies(namespaceName, p -> policies); log.info("[{}] Successfully updated backlog quota map: namespace={}, map={}", clientAppId(), namespaceName, jsonMapper().writeValueAsString(backlogQuota)); @@ -1468,8 +1428,7 @@ protected void internalRemoveBacklogQuota(BacklogQuotaType backlogQuotaType) { final BacklogQuotaType quotaType = backlogQuotaType != null ? backlogQuotaType : BacklogQuotaType.destination_storage; try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.backlog_quota_map.remove(quotaType); return policies; }); @@ -1487,8 +1446,8 @@ protected void internalSetRetention(RetentionPolicies retention) { validatePoliciesReadOnlyAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - Policies policies = namespaceResources().get(path).orElseThrow(() -> new RestException(Status.NOT_FOUND, + Policies policies = namespaceResources().getPolicies(namespaceName) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace policies does not exist")); if (!checkQuotas(policies, retention)) { log.warn("[{}] Failed to update retention configuration" @@ -1498,7 +1457,7 @@ protected void internalSetRetention(RetentionPolicies retention) { "Retention Quota must exceed configured backlog quota for namespace."); } policies.retention_policies = retention; - namespaceResources().set(path, p -> policies); + namespaceResources().setPolicies(namespaceName, p -> policies); log.info("[{}] Successfully updated retention configuration: namespace={}, map={}", clientAppId(), namespaceName, jsonMapper().writeValueAsString(retention)); } catch (RestException pfe) { @@ -1526,8 +1485,7 @@ protected void internalSetPersistence(PersistencePolicies persistence) { private void doUpdatePersistence(PersistencePolicies persistence) { try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies)->{ + updatePolicies(namespaceName, policies -> { policies.persistence = persistence; return policies; }); @@ -1749,8 +1707,7 @@ protected void internalSetSubscriptionAuthMode(SubscriptionAuthMode subscription SubscriptionAuthMode authMode = subscriptionAuthMode == null ? subscriptionAuthMode = SubscriptionAuthMode.None : subscriptionAuthMode; try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.subscription_auth_mode = authMode; return policies; }); @@ -1770,7 +1727,7 @@ protected void internalModifyEncryptionRequired(boolean encryptionRequired) { validatePoliciesReadOnlyAccess(); try { - updatePolicies(path(POLICIES, namespaceName.toString()), (policies) -> { + updatePolicies(namespaceName, policies -> { policies.encryption_required = encryptionRequired; return policies; }); @@ -1803,13 +1760,13 @@ protected void internalSetInactiveTopic(InactiveTopicPolicies inactiveTopicPolic protected void internalSetPolicies(String fieldName, Object value) { try { - final String path = path(POLICIES, namespaceName.toString()); - Policies policies = namespaceResources().get(path).orElseThrow(() -> new RestException(Status.NOT_FOUND, + Policies policies = namespaceResources().getPolicies(namespaceName) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace policies does not exist")); Field field = Policies.class.getDeclaredField(fieldName); field.setAccessible(true); field.set(policies, value); - namespaceResources().set(path, p -> policies); + namespaceResources().setPolicies(namespaceName, p -> policies); log.info("[{}] Successfully updated {} configuration: namespace={}, value={}", clientAppId(), fieldName, namespaceName, jsonMapper().writeValueAsString(value)); @@ -1840,8 +1797,7 @@ protected void internalSetNamespaceAntiAffinityGroup(String antiAffinityGroup) { } try { - String path = joinPath(LOCAL_POLICIES_ROOT, this.namespaceName.toString()); - getLocalPolicies().setWithCreate(path, (lp)-> + getLocalPolicies().setLocalPoliciesWithCreate(namespaceName, (lp)-> lp.map(policies -> new LocalPolicies(policies.bundles, policies.bookieAffinityGroup, antiAffinityGroup)) @@ -1862,7 +1818,7 @@ protected String internalGetNamespaceAntiAffinityGroup() { try { return getLocalPolicies() - .get(AdminResource.joinPath(LOCAL_POLICIES_ROOT, namespaceName.toString())) + .getLocalPolicies(namespaceName) .orElse(new LocalPolicies()).namespaceAntiAffinityGroup; } catch (Exception e) { log.error("[{}] Failed to get the antiAffinityGroup of namespace {}", clientAppId(), namespaceName, e); @@ -1877,8 +1833,7 @@ protected void internalRemoveNamespaceAntiAffinityGroup() { log.info("[{}] Deleting anti-affinity group for {}", clientAppId(), namespaceName); try { - final String path = joinPath(LOCAL_POLICIES_ROOT, namespaceName.toString()); - getLocalPolicies().set(path, (policies)-> + getLocalPolicies().setLocalPolicies(namespaceName, (policies)-> new LocalPolicies(policies.bundles, policies.bookieAffinityGroup, null)); @@ -1904,13 +1859,12 @@ protected List internalGetAntiAffinityNamespaces(String cluster, String validateClusterExists(cluster); try { - List namespaces = getListOfNamespaces(tenant); + List namespaces = tenantResources().getListOfNamespaces(tenant); return namespaces.stream().filter(ns -> { Optional policies; try { - policies = getLocalPolicies() - .get(AdminResource.joinPath(LOCAL_POLICIES_ROOT, ns)); + policies = getLocalPolicies().getLocalPolicies(namespaceName); } catch (Exception e) { throw new RuntimeException(e); } @@ -2010,7 +1964,7 @@ private void unsubscribe(NamespaceName nsName, String bundleRange, String subscr */ private void validatePeerClusterConflict(String clusterName, Set replicationClusters) { try { - ClusterData clusterData = clusterResources().get(path("clusters", clusterName)).orElseThrow( + ClusterData clusterData = clusterResources().getCluster(clusterName).orElseThrow( () -> new RestException(Status.PRECONDITION_FAILED, "Invalid replication cluster " + clusterName)); Set peerClusters = clusterData.getPeerClusterNames(); if (peerClusters != null && !peerClusters.isEmpty()) { @@ -2130,8 +2084,7 @@ protected void internalSetMaxProducersPerTopic(Integer maxProducersPerTopic) { throw new RestException(Status.PRECONDITION_FAILED, "maxProducersPerTopic must be 0 or more"); } - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies)->{ + updatePolicies(namespaceName, policies -> { policies.max_producers_per_topic = maxProducersPerTopic; return policies; }); @@ -2164,8 +2117,7 @@ protected void internalSetMaxConsumersPerTopic(Integer maxConsumersPerTopic) { if (maxConsumersPerTopic != null && maxConsumersPerTopic < 0) { throw new RestException(Status.PRECONDITION_FAILED, "maxConsumersPerTopic must be 0 or more"); } - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.max_consumers_per_topic = maxConsumersPerTopic; return policies; }); @@ -2195,8 +2147,7 @@ protected void internalSetMaxConsumersPerSubscription(Integer maxConsumersPerSub throw new RestException(Status.PRECONDITION_FAILED, "maxConsumersPerSubscription must be 0 or more"); } - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.max_consumers_per_subscription = maxConsumersPerSubscription; return policies; }); @@ -2224,8 +2175,7 @@ protected void internalSetMaxUnackedMessagesPerConsumer(Integer maxUnackedMessag "maxUnackedMessagesPerConsumer must be 0 or more"); } try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies)->{ + updatePolicies(namespaceName, policies -> { policies.max_unacked_messages_per_consumer = maxUnackedMessagesPerConsumer; return policies; }); @@ -2269,8 +2219,7 @@ protected void internalSetMaxUnackedMessagesPerSubscription(Integer maxUnackedMe "maxUnackedMessagesPerSubscription must be 0 or more"); } try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.max_unacked_messages_per_subscription = maxUnackedMessagesPerSubscription; return policies; }); @@ -2301,8 +2250,7 @@ protected void internalSetCompactionThreshold(Long newThreshold) { throw new RestException(Status.PRECONDITION_FAILED, "compactionThreshold must be 0 or more"); } - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { policies.compaction_threshold = newThreshold; return policies; }); @@ -2333,8 +2281,7 @@ protected void internalSetOffloadThreshold(long newThreshold) { validatePoliciesReadOnlyAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { if (policies.offload_policies == null) { policies.offload_policies = new OffloadPoliciesImpl(); } @@ -2368,8 +2315,7 @@ protected void internalSetOffloadDeletionLag(Long newDeletionLagMs) { validateNamespacePolicyOperation(namespaceName, PolicyName.OFFLOAD, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { if (policies.offload_policies == null) { policies.offload_policies = new OffloadPoliciesImpl(); } @@ -2499,10 +2445,9 @@ private void mutatePolicy(Function policyTransformation, Function getter, String policyName) { try { - final String path = path(POLICIES, namespaceName.toString()); MutableObject exception = new MutableObject(null); MutableObject policiesObj = new MutableObject(null); - updatePolicies(path, (policies) -> { + updatePolicies(namespaceName, policies -> { try { policies = policyTransformation.apply(policies); } catch (Exception e) { @@ -2531,8 +2476,7 @@ protected void internalSetOffloadPolicies(AsyncResponse asyncResponse, OffloadPo validateOffloadPolicies(offloadPolicies); try { - final String path = path(POLICIES, namespaceName.toString()); - namespaceResources().setAsync(path, (policies) -> { + namespaceResources().setPoliciesAsync(namespaceName, policies -> { if (Objects.equals(offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis(), OffloadPoliciesImpl.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS)) { offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(policies.offload_deletion_lag_ms); @@ -2570,8 +2514,7 @@ protected void internalRemoveOffloadPolicies(AsyncResponse asyncResponse) { validateNamespacePolicyOperation(namespaceName, PolicyName.OFFLOAD, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); try { - final String path = path(POLICIES, namespaceName.toString()); - namespaceResources().setAsync(path, (policies) -> { + namespaceResources().setPoliciesAsync(namespaceName, (policies) -> { policies.offload_policies = null; return policies; }).thenApply(r -> { @@ -2644,8 +2587,7 @@ protected void internalSetMaxTopicsPerNamespace(Integer maxTopicsPerNamespace) { protected void internalSetProperty(String key, String value, AsyncResponse asyncResponse) { validatePoliciesReadOnlyAccess(); - final String path = path(POLICIES, namespaceName.toString()); - updatePoliciesAsync(path, (policies) -> { + updatePoliciesAsync(namespaceName, policies -> { policies.properties.put(key, value); return policies; }).thenAccept(v -> { @@ -2663,8 +2605,7 @@ protected void internalSetProperty(String key, String value, AsyncResponse async protected void internalSetProperties(Map properties, AsyncResponse asyncResponse) { validatePoliciesReadOnlyAccess(); - final String path = path(POLICIES, namespaceName.toString()); - updatePoliciesAsync(path, (policies) -> { + updatePoliciesAsync(namespaceName, policies -> { policies.properties.putAll(properties); return policies; }).thenAccept(v -> { @@ -2705,9 +2646,9 @@ protected void internalGetProperties(AsyncResponse asyncResponse) { protected void internalRemoveProperty(String key, AsyncResponse asyncResponse) { validatePoliciesReadOnlyAccess(); - final String path = path(POLICIES, namespaceName.toString()); + AtomicReference oldVal = new AtomicReference<>(null); - updatePoliciesAsync(path, (policies) -> { + updatePoliciesAsync(namespaceName, policies -> { oldVal.set(policies.properties.remove(key)); return policies; }).thenAccept(v -> { @@ -2725,9 +2666,8 @@ protected void internalRemoveProperty(String key, AsyncResponse asyncResponse) { protected void internalClearProperties(AsyncResponse asyncResponse) { validatePoliciesReadOnlyAccess(); - final String path = path(POLICIES, namespaceName.toString()); AtomicReference clearedCount = new AtomicReference<>(0); - updatePoliciesAsync(path, (policies) -> { + updatePoliciesAsync(namespaceName, policies -> { clearedCount.set(policies.properties.size()); policies.properties.clear(); return policies; @@ -2744,11 +2684,11 @@ protected void internalClearProperties(AsyncResponse asyncResponse) { }); } - private CompletableFuture updatePoliciesAsync(String path, Function updateFunction) { + private CompletableFuture updatePoliciesAsync(NamespaceName ns, Function updateFunction) { CompletableFuture result = new CompletableFuture<>(); - namespaceResources().setAsync(path, updateFunction) + namespaceResources().setPoliciesAsync(ns, updateFunction) .thenAccept(v -> { - log.info("[{}] Successfully updated the {} on namespace {}", clientAppId(), path, namespaceName); + log.info("[{}] Successfully updated the policies on namespace {}", clientAppId(), namespaceName); result.complete(null); }) .exceptionally(ex -> { @@ -2768,10 +2708,10 @@ private CompletableFuture updatePoliciesAsync(String path, Function updateFunction) { + private void updatePolicies(NamespaceName ns, Function updateFunction) { // Force to read the data s.t. the watch to the cache content is setup. try { - updatePoliciesAsync(path, updateFunction).get(namespaceResources().getOperationTimeoutSec(), + updatePoliciesAsync(ns, updateFunction).get(namespaceResources().getOperationTimeoutSec(), TimeUnit.SECONDS); } catch (Exception e) { Throwable cause = e.getCause(); @@ -2788,10 +2728,9 @@ protected void internalSetNamespaceResourceGroup(String rgName) { validatePoliciesReadOnlyAccess(); if (rgName != null) { - final String resourceGroupPath = AdminResource.path(RESOURCEGROUPS, rgName); // check resourcegroup exists. try { - if (!resourceGroupResources().exists(resourceGroupPath)) { + if (!resourceGroupResources().resourceGroupExists(rgName)) { throw new RestException(Status.PRECONDITION_FAILED, "ResourceGroup does not exist"); } } catch (Exception e) { 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 d29fff6f33510..9000f3d8b5ea0 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 @@ -18,9 +18,7 @@ */ package org.apache.pulsar.broker.admin.impl; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.broker.resources.PulsarResources.DEFAULT_OPERATION_TIMEOUT_SEC; -import static org.apache.pulsar.common.util.Codec.decode; import com.fasterxml.jackson.core.JsonProcessingException; import com.github.zafarkhaja.semver.Version; import com.google.common.collect.Lists; @@ -66,7 +64,6 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.admin.AdminResource; -import org.apache.pulsar.broker.admin.ZkAdminPaths; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.service.BrokerServiceException.AlreadyRunningException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; @@ -150,7 +147,7 @@ protected List internalGetList() { // Validate that namespace exists, throws 404 if it doesn't exist try { - if (!namespaceResources().exists(path(POLICIES, namespaceName.toString()))) { + if (!namespaceResources().namespaceExists(namespaceName)) { throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); } } catch (RestException re) { @@ -160,24 +157,12 @@ protected List internalGetList() { throw new RestException(e); } - List topics = Lists.newArrayList(); - try { - String path = String.format("/managed-ledgers/%s/%s", namespaceName.toString(), domain()); - for (String topic : getLocalPolicies().getChildren(path)) { - if (domain().equals(TopicDomain.persistent.toString())) { - topics.add(TopicName.get(domain(), namespaceName, decode(topic)).toString()); - } - } - } catch (org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException e) { - // NoNode means there are no topics in this domain for this namespace + return topicResources().listPersistentTopicsAsync(namespaceName).join(); } catch (Exception e) { log.error("[{}] Failed to get topics list for namespace {}", clientAppId(), namespaceName, e); throw new RestException(e); } - - topics.sort(null); - return topics; } protected List internalGetPartitionedTopicList() { @@ -185,7 +170,7 @@ protected List internalGetPartitionedTopicList() { validateNamespaceOperation(namespaceName, NamespaceOperation.GET_TOPICS); // Validate that namespace exists, throws 404 if it doesn't exist try { - if (!namespaceResources().exists(path(POLICIES, namespaceName.toString()))) { + if (!namespaceResources().namespaceExists(namespaceName)) { log.warn("[{}] Failed to get partitioned topic list {}: Namespace does not exist", clientAppId(), namespaceName); throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); @@ -206,7 +191,7 @@ protected Map> internalGetPermissionsOnTopic() { String topicUri = topicName.toString(); try { - Policies policies = namespaceResources().get(path(POLICIES, namespaceName.toString())) + Policies policies = namespaceResources().getPolicies(namespaceName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); Map> permissions = Maps.newHashMap(); @@ -263,7 +248,7 @@ public void validateAdminOperationOnTopic(boolean authoritative) { private void grantPermissions(String topicUri, String role, Set actions) { try { - namespaceResources().set(path(POLICIES, namespaceName.toString()), (policies) -> { + namespaceResources().setPolicies(namespaceName, policies -> { if (!policies.auth_policies.getTopicAuthentication().containsKey(topicUri)) { policies.auth_policies.getTopicAuthentication().put(topicUri, new HashMap<>()); } @@ -317,7 +302,7 @@ protected void internalDeleteTopicForcefully(boolean authoritative, boolean dele private void revokePermissions(String topicUri, String role) { Policies policies; try { - policies = namespaceResources().get(path(POLICIES, namespaceName.toString())) + policies = namespaceResources().getPolicies(namespaceName) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); } catch (Exception e) { log.error("[{}] Failed to revoke permissions for topic {}", clientAppId(), topicUri, e); @@ -330,9 +315,8 @@ private void revokePermissions(String topicUri, String role) { throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the topic level"); } try { - // Write the new policies to zookeeper - String namespacePath = path(POLICIES, namespaceName.toString()); - namespaceResources().set(namespacePath, (p) -> { + // Write the new policies to metadata store + namespaceResources().setPolicies(namespaceName, p -> { p.auth_policies.getTopicAuthentication().get(topicUri).remove(role); return p; }); @@ -440,12 +424,12 @@ protected void internalUpdatePartitionedTopic(int numPartitions, // other clusters and then update number of partitions. if (!updateLocalTopicOnly) { CompletableFuture updatePartition = new CompletableFuture<>(); - final String path = ZkAdminPaths.partitionedTopicPath(topicName); updatePartitionInOtherCluster(numPartitions, clusters).thenRun(() -> { try { - namespaceResources().getPartitionedTopicResources().setAsync(path, (p) -> { - return new PartitionedTopicMetadata(numPartitions); - }).thenAccept(r -> updatePartition.complete(null)).exceptionally(ex -> { + namespaceResources().getPartitionedTopicResources() + .updatePartitionedTopicAsync(topicName, p -> + new PartitionedTopicMetadata(numPartitions) + ).thenAccept(r -> updatePartition.complete(null)).exceptionally(ex -> { updatePartition.completeExceptionally(ex.getCause()); return null; }); @@ -649,12 +633,11 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boole } } // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), - topicName.getEncodedLocalName()); try { - namespaceResources().getPartitionedTopicResources().deleteAsync(path).thenAccept(r2 -> { - log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName); - asyncResponse.resume(Response.noContent().build()); + namespaceResources().getPartitionedTopicResources() + .deletePartitionedTopicAsync(topicName).thenAccept(r2 -> { + log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName); + asyncResponse.resume(Response.noContent().build()); }).exceptionally(ex1 -> { log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, ex1.getCause()); if (ex1.getCause() @@ -997,10 +980,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut if (topicName.getDomain() == TopicDomain.persistent) { final Map> existsFutures = Maps.newConcurrentMap(); for (int i = 0; i < partitionMetadata.partitions; i++) { - String path = String.format("/managed-ledgers/%s/%s/%s", namespaceName.toString(), - domain(), topicName.getPartition(i).getEncodedLocalName()); - CompletableFuture exists = getLocalPolicies().existsAsync(path); - existsFutures.put(i, exists); + existsFutures.put(i, topicResources().persistentTopicExists(topicName.getPartition(i))); } FutureUtil.waitForAll(Lists.newArrayList(existsFutures.values())).thenApply(__ -> existsFutures.entrySet().stream().filter(e -> e.getValue().join().booleanValue()) @@ -1273,11 +1253,11 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean } } if (perPartition && stats.partitions.isEmpty()) { - String path = ZkAdminPaths.partitionedTopicPath(topicName); try { - boolean zkPathExists = namespaceResources().getPartitionedTopicResources().exists(path); - if (zkPathExists) { - stats.getPartitions().put(topicName.toString(), new TopicStatsImpl()); + boolean pathExists = namespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); + if (pathExists) { + stats.partitions.put(topicName.toString(), new TopicStatsImpl()); } else { asyncResponse.resume( new RestException(Status.NOT_FOUND, @@ -2629,7 +2609,7 @@ protected PersistentOfflineTopicStats internalGetBacklog(boolean authoritative) // Validate that namespace exists, throw 404 if it doesn't exist // note that we do not want to load the topic and hence skip authorization check try { - namespaceResources().get(path(POLICIES, namespaceName.toString())); + namespaceResources().getPolicies(namespaceName); } catch (org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException e) { log.warn("[{}] Failed to get topic backlog {}: Namespace does not exist", clientAppId(), namespaceName); throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); @@ -2676,12 +2656,10 @@ protected CompletableFuture> in if (applied && quotaMap.isEmpty()) { quotaMap = getNamespacePolicies(namespaceName).backlog_quota_map; if (quotaMap.isEmpty()) { - String namespace = namespaceName.toString(); for (BacklogQuota.BacklogQuotaType backlogQuotaType : BacklogQuota.BacklogQuotaType.values()) { quotaMap.put( backlogQuotaType, - namespaceBacklogQuota(namespace, - AdminResource.path(POLICIES, namespace), backlogQuotaType) + namespaceBacklogQuota(namespaceName, backlogQuotaType) ); } } @@ -3627,38 +3605,33 @@ private PersistentReplicator getReplicatorReference(String replName, PersistentT } private CompletableFuture updatePartitionedTopic(TopicName topicName, int numPartitions) { - final String path = ZkAdminPaths.partitionedTopicPath(topicName); - - CompletableFuture updatePartition = new CompletableFuture<>(); - createSubscriptions(topicName, numPartitions).thenAccept(res -> { - try { - namespaceResources().getPartitionedTopicResources().set(path, - p -> new PartitionedTopicMetadata(numPartitions)); - updatePartition.complete(null); - } catch (Exception e) { - getPartitionedTopicMetadataAsync(topicName, false, false).thenAccept(metadata -> { - int oldPartition = metadata.partitions; - for (int i = oldPartition; i < numPartitions; i++) { - String managedLedgerPath = ZkAdminPaths.managedLedgerPath(topicName.getPartition(i)); - namespaceResources().getPartitionedTopicResources() - .deleteAsync(managedLedgerPath).exceptionally(ex1 -> { - log.warn("[{}] Failed to clean up managedLedger znode {}", clientAppId(), - managedLedgerPath, ex1.getCause()); - return null; - }); - } - }).exceptionally(ex -> { - log.warn("[{}] Failed to clean up managedLedger znode", clientAppId(), ex.getCause()); - return null; + return createSubscriptions(topicName, numPartitions) + .thenCompose(__ -> { + CompletableFuture future = namespaceResources().getPartitionedTopicResources() + .updatePartitionedTopicAsync(topicName, + p -> new PartitionedTopicMetadata(numPartitions)); + future.exceptionally(ex -> { + // If the update operation fails, clean up the partitions that were created + getPartitionedTopicMetadataAsync(topicName, false, false) + .thenAccept(metadata -> { + int oldPartition = metadata.partitions; + for (int i = oldPartition; i < numPartitions; i++) { + topicResources().deletePersistentTopicAsync(topicName.getPartition(i)) + .exceptionally(ex1 -> { + log.warn("[{}] Failed to clean up managedLedger {}", + clientAppId(), + topicName, ex1.getCause()); + return null; + }); + } + }).exceptionally(e -> { + log.warn("[{}] Failed to clean up managedLedger", topicName, e); + return null; + }); + return null; + }); + return future; }); - updatePartition.completeExceptionally(e); - } - }).exceptionally(ex -> { - updatePartition.completeExceptionally(ex); - return null; - }); - - return updatePartition; } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceGroupsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceGroupsBase.java index 4e676a407d043..71fb6aae04476 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceGroupsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceGroupsBase.java @@ -18,9 +18,6 @@ */ package org.apache.pulsar.broker.admin.impl; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.RESOURCEGROUPS; -import java.util.Iterator; import java.util.List; import javax.ws.rs.core.Response; import org.apache.pulsar.broker.admin.AdminResource; @@ -29,7 +26,6 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.ResourceGroup; import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,10 +33,7 @@ public abstract class ResourceGroupsBase extends AdminResource { protected List internalGetResourceGroups() { try { validateSuperUserAccess(); - return getListOfResourcegroups("abc"); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get ResourceGroups list ", clientAppId()); - throw new RestException(Response.Status.NOT_FOUND, "Property does not exist"); + return resourceGroupResources().listResourceGroups(); } catch (Exception e) { log.error("[{}] Failed to get ResourceGroups list: {}", clientAppId(), e); throw new RestException(e); @@ -49,9 +42,8 @@ protected List internalGetResourceGroups() { protected ResourceGroup internalGetResourceGroup(String rgName) { try { - final String resourceGroupPath = AdminResource.path(RESOURCEGROUPS, rgName); validateSuperUserAccess(); - ResourceGroup resourceGroup = resourceGroupResources().get(resourceGroupPath) + ResourceGroup resourceGroup = resourceGroupResources().getResourceGroup(rgName) .orElseThrow(() -> new RestException(Response.Status.NOT_FOUND, "ResourceGroup does not exist")); return resourceGroup; } catch (RestException re) { @@ -63,10 +55,9 @@ protected ResourceGroup internalGetResourceGroup(String rgName) { } protected void internalUpdateResourceGroup(String rgName, ResourceGroup rgConfig) { - final String resourceGroupPath = AdminResource.path(RESOURCEGROUPS, rgName); try { - ResourceGroup resourceGroup = resourceGroupResources().get(resourceGroupPath).orElseThrow(() -> + ResourceGroup resourceGroup = resourceGroupResources().getResourceGroup(rgName).orElseThrow(() -> new RestException(Response.Status.NOT_FOUND, "ResourceGroup does not exist")); /* @@ -78,7 +69,7 @@ protected void internalUpdateResourceGroup(String rgName, ResourceGroup rgConfig resourceGroup.setDispatchRateInBytes(rgConfig.getDispatchRateInBytes()); // write back the new ResourceGroup config. - resourceGroupResources().set(resourceGroupPath, r -> resourceGroup); + resourceGroupResources().updateResourceGroup(rgName, r -> resourceGroup); log.info("[{}] Successfully updated the ResourceGroup {}", clientAppId(), rgName); } catch (RestException pfe) { throw pfe; @@ -89,9 +80,8 @@ protected void internalUpdateResourceGroup(String rgName, ResourceGroup rgConfig } protected void internalCreateResourceGroup(String rgName, ResourceGroup rgConfig) { - final String resourceGroupPath = AdminResource.path(RESOURCEGROUPS, rgName); try { - resourceGroupResources().create(resourceGroupPath, rgConfig); + resourceGroupResources().createResourceGroup(rgName, rgConfig); log.info("[{}] Created ResourceGroup {}", clientAppId(), rgName); } catch (MetadataStoreException.AlreadyExistsException e) { log.warn("[{}] Failed to create ResourceGroup {} - already exists", clientAppId(), rgName); @@ -109,10 +99,9 @@ protected void internalCreateOrUpdateResourceGroup(String rgName, ResourceGroup /* * see if ResourceGroup exists and treat the request as a update if it does. */ - final String resourceGroupPath = AdminResource.path(RESOURCEGROUPS, rgName); boolean rgExists = false; try { - rgExists = resourceGroupResources().exists(resourceGroupPath); + rgExists = resourceGroupResources().resourceGroupExists(rgName); } catch (Exception e) { log.error("[{}] Failed to create/update ResourceGroup {}: {}", clientAppId(), rgName, e); } @@ -134,16 +123,9 @@ protected void internalCreateOrUpdateResourceGroup(String rgName, ResourceGroup } protected boolean internalCheckRgInUse(String rgName) { - List tenants; try { - tenants = tenantResources().getChildren(path(POLICIES)); - Iterator tenantsIterator = tenants.iterator(); - while (tenantsIterator.hasNext()) { - String tenant = tenantsIterator.next(); - List namespaces = getListOfNamespaces(tenant); - Iterator namespaceIterator = namespaces.iterator(); - while (namespaceIterator.hasNext()) { - String namespace = namespaceIterator.next(); + for (String tenant : tenantResources().listTenants()) { + for (String namespace : tenantResources().getListOfNamespaces(tenant)) { Policies policies = getNamespacePolicies(NamespaceName.get(namespace)); if (null != policies && rgName.equals(policies.resource_group_name)) { return true; @@ -169,8 +151,7 @@ protected void internalDeleteResourceGroup(String rgName) { if (internalCheckRgInUse(rgName)) { throw new RestException(Response.Status.PRECONDITION_FAILED, "ResourceGroup is in use"); } - final String globalZkResourceGroupPath = path(RESOURCEGROUPS, rgName); - resourceGroupResources().delete(globalZkResourceGroupPath); + resourceGroupResources().deleteResourceGroup(rgName); log.info("[{}] Deleted ResourceGroup {}", clientAppId(), rgName); } catch (Exception e) { log.error("[{}] Failed to delete ResourceGroup {}.", clientAppId(), rgName, e); 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 7d3042b64113a..978e59eb737c0 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 @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.admin.impl; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import com.google.common.collect.Lists; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiParam; @@ -71,7 +70,7 @@ public void getTenants(@Suspended final AsyncResponse asyncResponse) { asyncResponse.resume(e); return; } - tenantResources().getChildrenAsync(path(POLICIES)).whenComplete((tenants, e) -> { + tenantResources().listTenantsAsync().whenComplete((tenants, e) -> { if (e != null) { log.error("[{}] Failed to get tenants list", clientAppId, e); asyncResponse.resume(new RestException(e)); @@ -98,7 +97,7 @@ public void getTenantAdmin(@Suspended final AsyncResponse asyncResponse, asyncResponse.resume(e); } - tenantResources().getAsync(path(POLICIES, tenant)).whenComplete((tenantInfo, e) -> { + tenantResources().getTenantAsync(tenant).whenComplete((tenantInfo, e) -> { if (e != null) { log.error("[{}] Failed to get Tenant {}", clientAppId, e.getMessage()); asyncResponse.resume(new RestException(Status.INTERNAL_SERVER_ERROR, "Failed to get Tenant")); @@ -137,7 +136,7 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, return; } - tenantResources().getChildrenAsync(path(POLICIES)).whenComplete((tenants, e) -> { + tenantResources().listTenantsAsync().whenComplete((tenants, e) -> { if (e != null) { log.error("[{}] Failed to create tenant ", clientAppId, e.getCause()); asyncResponse.resume(new RestException(e)); @@ -154,12 +153,12 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, return; } } - tenantResources().existsAsync(path(POLICIES, tenant)).thenAccept(exist ->{ + tenantResources().tenantExistsAsync(tenant).thenAccept(exist ->{ if (exist) { asyncResponse.resume(new RestException(Status.CONFLICT, "Tenant already exist")); return; } - tenantResources().createAsync(path(POLICIES, tenant), tenantInfo).thenAccept((r) -> { + tenantResources().createTenantAsync(tenant, tenantInfo).thenAccept((r) -> { log.info("[{}] Created tenant {}", clientAppId(), tenant); asyncResponse.resume(Response.noContent().build()); }).exceptionally(ex -> { @@ -197,7 +196,7 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, } final String clientAddId = clientAppId(); - tenantResources().getAsync(path(POLICIES, tenant)).thenAccept(tenantAdmin -> { + tenantResources().getTenantAsync(tenant).thenAccept(tenantAdmin -> { if (!tenantAdmin.isPresent()) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Tenant " + tenant + " not found")); return; @@ -205,9 +204,7 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, TenantInfo oldTenantAdmin = tenantAdmin.get(); Set newClusters = new HashSet<>(newTenantAdmin.getAllowedClusters()); canUpdateCluster(tenant, oldTenantAdmin.getAllowedClusters(), newClusters).thenApply(r -> { - tenantResources().setAsync(path(POLICIES, tenant), old -> { - return newTenantAdmin; - }).thenAccept(done -> { + tenantResources().updateTenantAsync(tenant, old -> newTenantAdmin).thenAccept(done -> { log.info("Successfully updated tenant info {}", tenant); asyncResponse.resume(Response.noContent().build()); }).exceptionally(ex -> { @@ -256,29 +253,19 @@ protected void internalDeleteTenant(AsyncResponse asyncResponse, String tenant, } protected void internalDeleteTenant(AsyncResponse asyncResponse, String tenant) { - tenantResources().existsAsync(path(POLICIES, tenant)).thenApply(exists -> { + tenantResources().tenantExistsAsync(tenant).thenApply(exists -> { if (!exists) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Tenant doesn't exist")); return null; } return hasActiveNamespace(tenant).thenAccept(ns -> { try { - // already fetched children and they should be in the cache - List> clusterList = Lists.newArrayList(); - for (String cluster : tenantResources().getChildrenAsync(path(POLICIES, tenant)).get()) { - clusterList.add(tenantResources().deleteAsync(path(POLICIES, tenant, cluster))); - } - FutureUtil.waitForAll(clusterList).thenAccept(c -> { - tenantResources().deleteAsync(path(POLICIES, tenant)).thenAccept(t -> { - log.info("[{}] Deleted tenant {}", clientAppId(), tenant); - asyncResponse.resume(Response.noContent().build()); - }).exceptionally(ex -> { - log.error("Failed to delete tenant {}", tenant, ex.getCause()); - asyncResponse.resume(new RestException(ex)); - return null; - }); - }).exceptionally(ex -> { - log.error("Failed to delete clusters under tenant {}", tenant, ex.getCause()); + tenantResources().deleteTenantAsync(tenant) + .thenAccept(t -> { + log.info("[{}] Deleted tenant {}", clientAppId(), tenant); + asyncResponse.resume(Response.noContent().build()); + }).exceptionally(ex -> { + log.error("Failed to delete tenant {}", tenant, ex.getCause()); asyncResponse.resume(new RestException(ex)); return null; }); @@ -304,7 +291,7 @@ protected void internalDeleteTenantForcefully(AsyncResponse asyncResponse, Strin List namespaces; try { - namespaces = getListOfNamespaces(tenant); + namespaces = tenantResources().getListOfNamespaces(tenant); } catch (Exception e) { log.error("[{}] Failed to get namespaces list of {}", clientAppId(), tenant, e); asyncResponse.resume(new RestException(e)); @@ -334,7 +321,7 @@ protected void internalDeleteTenantForcefully(AsyncResponse asyncResponse, Strin try { - pulsar().getPulsarResources().getTopicResources().clearTennantPersistence(tenant).get(); + pulsar().getPulsarResources().getTopicResources().clearTenantPersistence(tenant).get(); } catch (ExecutionException | InterruptedException e) { // warn level log here since this failure has no side effect besides left a un-used metadata // and also will not affect the re-creation of tenant 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 529b8ce0cbe62..03849ec54cc3d 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 @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.admin.v1; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.common.policies.data.PoliciesUtil.getBundles; import com.google.common.collect.Lists; import io.swagger.annotations.Api; @@ -94,19 +93,19 @@ public List getTenantNamespaces(@PathParam("property") String property) @ApiOperation(hidden = true, value = "Get the list of all the namespaces for a certain property on single cluster.", response = String.class, responseContainer = "Set") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster doesn't exist")}) - public List getNamespacesForCluster(@PathParam("property") String property, + @ApiResponse(code = 404, message = "Tenant or cluster doesn't exist")}) + public List getNamespacesForCluster(@PathParam("property") String tenant, @PathParam("cluster") String cluster) { - validateTenantOperation(property, TenantOperation.LIST_NAMESPACES); + validateTenantOperation(tenant, TenantOperation.LIST_NAMESPACES); List namespaces = Lists.newArrayList(); if (!clusters().contains(cluster)) { - log.warn("[{}] Failed to get namespace list for property: {}/{} - Cluster does not exist", clientAppId(), - property, cluster); + log.warn("[{}] Failed to get namespace list for tenant: {}/{} - Cluster does not exist", clientAppId(), + tenant, cluster); throw new RestException(Status.NOT_FOUND, "Cluster does not exist"); } try { - for (String namespace : clusterResources().getChildren(path(POLICIES, property, cluster))) { - namespaces.add(String.format("%s/%s/%s", property, cluster, namespace)); + for (String namespace : clusterResources().getNamespacesForCluster(tenant, cluster)) { + namespaces.add(NamespaceName.get(tenant, cluster, namespace).toString()); } } catch (NotFoundException e) { // NoNode means there are no namespaces for this property on the specified cluster, returning empty list diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index d390049b40576..3f482f84add5b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -48,7 +48,6 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; import org.apache.pulsar.broker.PulsarServerException; -import org.apache.pulsar.broker.admin.ZkAdminPaths; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; import org.apache.pulsar.broker.web.RestException; @@ -288,10 +287,10 @@ public void getPartitionedStats( } } if (perPartition && stats.partitions.isEmpty()) { - String path = ZkAdminPaths.partitionedTopicPath(topicName); try { - boolean zkPathExists = namespaceResources().getPartitionedTopicResources().exists(path); - if (zkPathExists) { + boolean topicExists = namespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); + if (topicExists) { stats.getPartitions().put(topicName.toString(), new NonPersistentTopicStatsImpl()); } else { asyncResponse.resume( diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index bf652dfb7208f..a19ccc0dfbb0b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -23,11 +23,9 @@ import static java.lang.String.format; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.pulsar.broker.admin.AdminResource.PARTITIONED_TOPIC_PATH_ZNODE; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; import static org.apache.pulsar.broker.web.PulsarWebResource.joinPath; -import static org.apache.pulsar.common.util.Codec.decode; import com.google.common.collect.Lists; import com.google.common.hash.Hashing; import io.netty.channel.EventLoopGroup; @@ -315,7 +313,7 @@ public void registerBootstrapNamespaces() throws PulsarServerException { // we may not need strict ownership checking for bootstrap names for now for (String namespace : config.getBootstrapNamespaces()) { - if (registerNamespace(namespace, false)) { + if (registerNamespace(NamespaceName.get(namespace), false)) { LOG.info("added bootstrap namespace name in local cache: ns={}", namespace); } } @@ -324,19 +322,17 @@ public void registerBootstrapNamespaces() throws PulsarServerException { /** * Tried to registers a namespace to this instance. * - * @param namespace + * @param nsname * @param ensureOwned * @return * @throws PulsarServerException * @throws Exception */ - public boolean registerNamespace(String namespace, boolean ensureOwned) throws PulsarServerException { + public boolean registerNamespace(NamespaceName nsname, boolean ensureOwned) throws PulsarServerException { String myUrl = pulsar.getSafeBrokerServiceUrl(); try { - NamespaceName nsname = NamespaceName.get(namespace); - String otherUrl = null; NamespaceBundle nsFullBundle = null; @@ -354,7 +350,7 @@ public boolean registerNamespace(String namespace, boolean ensureOwned) throws P } String msg = String.format("namespace already owned by other broker : ns=%s expected=%s actual=%s", - namespace, myUrl, otherUrl); + nsname, myUrl, otherUrl); // ignore if not be owned for now if (!ensureOwned) { @@ -1128,23 +1124,19 @@ public CompletableFuture> getAllPartitions(NamespaceName namespaceN public CompletableFuture> getPartitions(NamespaceName namespaceName, TopicDomain topicDomain) { - String path = PulsarWebResource.path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), - topicDomain.toString()); - if (LOG.isDebugEnabled()) { - LOG.debug("Getting children from partitioned-topics now: {}", path); + LOG.debug("Getting children from partitioned-topics now: {} - {}", namespaceName, topicDomain); } - return pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().getChildrenAsync(path) + return pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .listPartitionedTopicsAsync(namespaceName, topicDomain) .thenCompose(topics -> { CompletableFuture> result = new CompletableFuture<>(); List resultPartitions = Collections.synchronizedList(Lists.newArrayList()); if (CollectionUtils.isNotEmpty(topics)) { List>> futures = Lists.newArrayList(); for (String topic : topics) { - String partitionedTopicName = String.format("%s://%s/%s", topicDomain.value(), - namespaceName.toString(), decode(topic)); - CompletableFuture> future = getPartitionsForTopic(TopicName.get(partitionedTopicName)); + CompletableFuture> future = getPartitionsForTopic(TopicName.get(topic)); futures.add(future); future.thenAccept(resultPartitions::addAll); } @@ -1279,11 +1271,11 @@ public CompletableFuture> getOwnerAsync(Namespa public void unloadSLANamespace() throws Exception { PulsarAdmin adminClient = null; - String namespaceName = getSLAMonitorNamespace(host, config); + NamespaceName namespaceName = getSLAMonitorNamespace(host, config); LOG.info("Checking owner for SLA namespace {}", namespaceName); - NamespaceBundle nsFullBundle = getFullBundle(NamespaceName.get(namespaceName)); + NamespaceBundle nsFullBundle = getFullBundle(namespaceName); if (!getOwner(nsFullBundle).isPresent()) { // No one owns the namespace so no point trying to unload it // Next lookup will assign the bundle to this broker. @@ -1292,38 +1284,38 @@ public void unloadSLANamespace() throws Exception { LOG.info("Trying to unload SLA namespace {}", namespaceName); adminClient = pulsar.getAdminClient(); - adminClient.namespaces().unload(namespaceName); + adminClient.namespaces().unload(namespaceName.toString()); LOG.info("Namespace {} unloaded successfully", namespaceName); } - public static String getHeartbeatNamespace(String host, ServiceConfiguration config) { + public static NamespaceName getHeartbeatNamespace(String host, ServiceConfiguration config) { Integer port = null; if (config.getWebServicePort().isPresent()) { port = config.getWebServicePort().get(); } else if (config.getWebServicePortTls().isPresent()) { port = config.getWebServicePortTls().get(); } - return String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), host, port); + return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), host, port)); } - public static String getHeartbeatNamespaceV2(String host, ServiceConfiguration config) { + public static NamespaceName getHeartbeatNamespaceV2(String host, ServiceConfiguration config) { Integer port = null; if (config.getWebServicePort().isPresent()) { port = config.getWebServicePort().get(); } else if (config.getWebServicePortTls().isPresent()) { port = config.getWebServicePortTls().get(); } - return String.format(HEARTBEAT_NAMESPACE_FMT_V2, host, port); + return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, host, port)); } - public static String getSLAMonitorNamespace(String host, ServiceConfiguration config) { + public static NamespaceName getSLAMonitorNamespace(String host, ServiceConfiguration config) { Integer port = null; if (config.getWebServicePort().isPresent()) { port = config.getWebServicePort().get(); } else if (config.getWebServicePortTls().isPresent()) { port = config.getWebServicePortTls().get(); } - return String.format(SLA_NAMESPACE_FMT, config.getClusterName(), host, port); + return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), host, port)); } public static String checkHeartbeatNamespace(ServiceUnitId ns) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java index b924f7b5e1020..48df56f80e156 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java @@ -18,10 +18,9 @@ */ package org.apache.pulsar.broker.resourcegroup; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.RESOURCEGROUPS; -import static org.apache.pulsar.common.policies.path.PolicyPath.path; import com.google.common.collect.Sets; import java.util.HashSet; +import java.util.Optional; import java.util.Set; import java.util.function.Consumer; import org.apache.pulsar.broker.PulsarService; @@ -61,7 +60,7 @@ public ResourceGroupConfigListener(ResourceGroupService rgService, PulsarService } private void loadAllResourceGroups() { - rgResources.getChildrenAsync(path(RESOURCEGROUPS)).whenCompleteAsync((rgList, ex) -> { + rgResources.listResourceGroupsAsync().whenCompleteAsync((rgList, ex) -> { if (ex != null) { LOG.error("Exception when fetching resource groups", ex); return; @@ -81,9 +80,8 @@ private void loadAllResourceGroups() { final Sets.SetView addList = Sets.difference(newSet, existingSet); for (String rgName: addList) { - final String resourceGroupPath = path(RESOURCEGROUPS, rgName); pulsarService.getPulsarResources().getResourcegroupResources() - .getAsync(resourceGroupPath).thenAcceptAsync((optionalRg) -> { + .getResourceGroupAsync(rgName).thenAcceptAsync(optionalRg -> { ResourceGroup rg = optionalRg.get(); createResourceGroup(rgName, rg); }).exceptionally((ex1) -> { @@ -116,10 +114,8 @@ public synchronized void createResourceGroup(String rgName, ResourceGroup rg) { } } - private void updateResourceGroup(String notifyPath) { - String rgName = notifyPath.substring(notifyPath.lastIndexOf('/') + 1); - - rgResources.getAsync(notifyPath).whenComplete((optionalRg, ex) -> { + private void updateResourceGroup(String rgName) { + rgResources.getResourceGroupAsync(rgName).whenComplete((optionalRg, ex) -> { if (ex != null) { LOG.error("Exception when getting resource group {}", rgName, ex); return; @@ -138,19 +134,19 @@ private void updateResourceGroup(String notifyPath) { public void accept(Notification notification) { String notifyPath = notification.getPath(); - if (!notifyPath.startsWith(path(RESOURCEGROUPS))) { + if (!ResourceGroupResources.isResourceGroupPath(notifyPath)) { return; } LOG.info("Metadata store notification: Path {}, Type {}", notifyPath, notification.getType()); - String rgName = notifyPath.substring(notifyPath.lastIndexOf('/') + 1); + Optional rgName = ResourceGroupResources.resourceGroupNameFromPath(notifyPath); if ((notification.getType() == NotificationType.ChildrenChanged) || (notification.getType() == NotificationType.Created)) { loadAllResourceGroups(); - } else if (!RESOURCEGROUPS.equals(rgName)) { + } else if (rgName.isPresent()) { switch (notification.getType()) { case Modified: - updateResourceGroup(notifyPath); + updateResourceGroup(rgName.get()); break; default: break; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupNamespaceConfigListener.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupNamespaceConfigListener.java index 148973786d078..e0d62f15b5ef5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupNamespaceConfigListener.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupNamespaceConfigListener.java @@ -18,13 +18,12 @@ */ package org.apache.pulsar.broker.resourcegroup; -import static org.apache.pulsar.broker.admin.ZkAdminPaths.POLICIES; -import static org.apache.pulsar.common.policies.path.PolicyPath.path; import java.util.function.Consumer; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.resources.TenantResources; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; @@ -60,10 +59,8 @@ public ResourceGroupNamespaceConfigListener(ResourceGroupService rgService, Puls this.namespaceResources.getStore().registerListener(this); } - private void updateNamespaceResourceGroup(String path) { - String nsName = path.substring(path(POLICIES).length() + 1); - - namespaceResources.getAsync(path).whenCompleteAsync((optionalPolicies, ex) -> { + private void updateNamespaceResourceGroup(NamespaceName nsName) { + namespaceResources.getPoliciesAsync(nsName).whenCompleteAsync((optionalPolicies, ex) -> { if (ex != null) { LOG.error("Exception when getting namespace {}", nsName, ex); return; @@ -74,18 +71,24 @@ private void updateNamespaceResourceGroup(String path) { } private void loadAllNamespaceResourceGroups() { - tenantResources.getChildrenAsync(path(POLICIES)).whenComplete((tenantList, ex) -> { + tenantResources.listTenantsAsync().whenComplete((tenantList, ex) -> { if (ex != null) { LOG.error("Exception when fetching tenants", ex); return; } for (String ts: tenantList) { - namespaceResources.getChildrenAsync(path(POLICIES, ts)).whenComplete((nsList, ex1) -> { + namespaceResources.listNamespacesAsync(ts).whenComplete((nsList, ex1) -> { if (ex1 != null) { LOG.error("Exception when fetching namespaces", ex1); } else { - for (String ns: nsList) { - updateNamespaceResourceGroup(path(POLICIES, ts, ns)); + for (String ns : nsList) { + NamespaceName nsn = NamespaceName.get(ts, ns); + namespaceResources.namespaceExistsAsync(nsn) + .thenAccept(exists -> { + if (exists) { + updateNamespaceResourceGroup(NamespaceName.get(ts, ns)); + } + }); } } }); @@ -97,7 +100,7 @@ public void reloadAllNamespaceResourceGroups() { loadAllNamespaceResourceGroups(); } - public void reconcileNamespaceResourceGroup(String ns, Policies policy) { + public void reconcileNamespaceResourceGroup(NamespaceName ns, Policies policy) { boolean delete = false, add = false; org.apache.pulsar.broker.resourcegroup.ResourceGroup current = rgService .getNamespaceResourceGroup(ns); @@ -133,7 +136,7 @@ public void reconcileNamespaceResourceGroup(String ns, Policies policy) { public void accept(Notification notification) { String notifyPath = notification.getPath(); - if (!notifyPath.startsWith(path(POLICIES))) { + if (!NamespaceResources.pathIsFromNamespace(notifyPath)) { return; } String[] parts = notifyPath.split("/"); @@ -148,21 +151,24 @@ public void accept(Notification notification) { reloadAllNamespaceResourceGroups(); } else if (parts.length == 5) { switch (notification.getType()) { - case Modified: - updateNamespaceResourceGroup(notifyPath); - break; - case Deleted: - String nsName = notifyPath.substring(path(POLICIES).length() + 1); - ResourceGroup rg = rgService - .getNamespaceResourceGroup(nsName); - if (rg != null) { - try { - rgService.unRegisterNameSpace(rg.resourceGroupName, nsName); - } catch (PulsarAdminException e) { - LOG.error("Failed to unregister namespace", e); + case Modified: { + NamespaceName nsName = NamespaceResources.namespaceFromPath(notifyPath); + updateNamespaceResourceGroup(nsName); + break; + } + case Deleted: { + NamespaceName nsName = NamespaceResources.namespaceFromPath(notifyPath); + ResourceGroup rg = rgService + .getNamespaceResourceGroup(nsName); + if (rg != null) { + try { + rgService.unRegisterNameSpace(rg.resourceGroupName, nsName); + } catch (PulsarAdminException e) { + LOG.error("Failed to unregister namespace", e); + } } + break; } - break; default: break; } 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 a7dbf3747d1c8..76c8c20797002 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 @@ -243,15 +243,7 @@ public void unRegisterTenant(String resourceGroupName, String tenantName) throws * @param fqNamespaceName (i.e., in "tenant/Namespace" format) * @throws if the RG does not exist, or if the NS already references the RG. */ - public void registerNameSpace(String resourceGroupName, String fqNamespaceName) throws PulsarAdminException { - // Check that it is a fully qualified NS (i.e., expect a '/' in it). - try { - val checkFqName = NamespaceName.get(fqNamespaceName); - } catch (RuntimeException rte) { - String errMesg = "Unexpected format found in fully-qualified namespace " + fqNamespaceName; - throw new PulsarAdminException(errMesg); - } - + public void registerNameSpace(String resourceGroupName, NamespaceName fqNamespaceName) throws PulsarAdminException { ResourceGroup rg = checkResourceGroupExists(resourceGroupName); // Check that the NS-name doesn't already have a RG association. @@ -262,8 +254,8 @@ public void registerNameSpace(String resourceGroupName, String fqNamespaceName) throw new PulsarAdminException(errMesg); } - ResourceGroupOpStatus status = rg.registerUsage(fqNamespaceName, ResourceGroupRefTypes.Namespaces, true, - this.resourceUsageTransportManagerMgr); + ResourceGroupOpStatus status = rg.registerUsage(fqNamespaceName.toString(), ResourceGroupRefTypes.Namespaces, + true, this.resourceUsageTransportManagerMgr); if (status == ResourceGroupOpStatus.Exists) { String errMesg = String.format("Namespace {} already references the target resource group {}", fqNamespaceName, resourceGroupName); @@ -282,19 +274,12 @@ public void registerNameSpace(String resourceGroupName, String fqNamespaceName) * @param fqNamespaceName i.e., in "tenant/Namespace" format) * @throws if the RG does not exist, or if the NS does not references the RG yet. */ - public void unRegisterNameSpace(String resourceGroupName, String fqNamespaceName) throws PulsarAdminException { - // Check that it is a fully qualified NS (i.e., expect a '/' in it). - try { - val checkFqName = NamespaceName.get(fqNamespaceName); - } catch (RuntimeException rte) { - String errMesg = "Unexpected format found in fully-qualified namespace " + fqNamespaceName; - throw new PulsarAdminException(errMesg); - } - + public void unRegisterNameSpace(String resourceGroupName, NamespaceName fqNamespaceName) + throws PulsarAdminException { ResourceGroup rg = checkResourceGroupExists(resourceGroupName); - ResourceGroupOpStatus status = rg.registerUsage(fqNamespaceName, ResourceGroupRefTypes.Namespaces, false, - this.resourceUsageTransportManagerMgr); + ResourceGroupOpStatus status = rg.registerUsage(fqNamespaceName.toString(), ResourceGroupRefTypes.Namespaces, + false, this.resourceUsageTransportManagerMgr); if (status == ResourceGroupOpStatus.DoesNotExist) { String errMesg = String.format("Namespace {} does not yet reference resource group {}", fqNamespaceName, resourceGroupName); @@ -312,7 +297,7 @@ public void unRegisterNameSpace(String resourceGroupName, String fqNamespaceName * @param namespaceName * @throws if the RG does not exist, or if the NS already references the RG. */ - public ResourceGroup getNamespaceResourceGroup(String namespaceName) { + public ResourceGroup getNamespaceResourceGroup(NamespaceName namespaceName) { return this.namespaceToRGsMap.get(namespaceName); } @@ -550,7 +535,7 @@ protected void aggregateResourceGroupLocalUsages() { final TopicName topic = TopicName.get(topicName); final String tenantString = topic.getTenant(); final String nsString = topic.getNamespacePortion(); - final String fqNamespace = topic.getNamespace(); + final NamespaceName fqNamespace = topic.getNamespaceObject(); // Can't use containsKey here, as that checks for exact equality // (we need a check for string-comparison). @@ -743,7 +728,7 @@ private void checkRGCreateParams(String rgName, org.apache.pulsar.common.policie private ConcurrentHashMap tenantToRGsMap = new ConcurrentHashMap<>(); // Given a qualified NS-name (i.e., in "tenant/namespace" format), record its associated resource-group - private ConcurrentHashMap namespaceToRGsMap = new ConcurrentHashMap<>(); + private ConcurrentHashMap namespaceToRGsMap = new ConcurrentHashMap<>(); // Maps to maintain the usage per topic, in produce/consume directions. private ConcurrentHashMap topicProduceStats = new ConcurrentHashMap<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index b7fdb245d2c93..b7e85dadb2464 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -18,13 +18,11 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.web.PulsarWebResource.path; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.PulsarServerException; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -246,13 +244,11 @@ public static String getReplicatorName(String replicatorPrefix, String cluster) */ private void validatePartitionedTopic(String topic, BrokerService brokerService) throws NamingException { TopicName topicName = TopicName.get(topic); - String partitionedTopicPath = path(AdminResource.PARTITIONED_TOPIC_PATH_ZNODE, - topicName.getNamespace(), topicName.getDomain().toString(), - topicName.getEncodedLocalName()); boolean isPartitionedTopic = false; try { - isPartitionedTopic = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(partitionedTopicPath).isPresent(); + isPartitionedTopic = + brokerService.pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); } catch (Exception e) { log.warn("Failed to verify partitioned topic {}-{}", topicName, e.getMessage()); } 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 87ee912173430..042f9ff0b3ffa 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 @@ -18,37 +18,34 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import com.google.common.collect.Lists; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +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.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; -import org.slf4j.LoggerFactory; +@Slf4j public class BacklogQuotaManager { - private static final Logger log = LoggerFactory.getLogger(BacklogQuotaManager.class); private final BacklogQuotaImpl defaultQuota; - private final ZooKeeperDataCache zkCache; private final PulsarService pulsar; private final boolean isTopicLevelPoliciesEnable; + private final NamespaceResources namespaceResources; public BacklogQuotaManager(PulsarService pulsar) { @@ -60,7 +57,7 @@ public BacklogQuotaManager(PulsarService pulsar) { .limitTime(pulsar.getConfiguration().getBacklogQuotaDefaultLimitSecond()) .retentionPolicy(pulsar.getConfiguration().getBacklogQuotaDefaultRetentionPolicy()) .build(); - this.zkCache = pulsar.getConfigurationCache().policiesCache(); + this.namespaceResources = pulsar.getPulsarResources().getNamespaceResources(); this.pulsar = pulsar; } @@ -68,9 +65,9 @@ public BacklogQuotaImpl getDefaultQuota() { return this.defaultQuota; } - public BacklogQuotaImpl getBacklogQuota(String namespace, String policyPath, BacklogQuotaType backlogQuotaType) { + public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaType backlogQuotaType) { try { - return zkCache.get(policyPath) + return namespaceResources.getPolicies(namespace) .map(p -> (BacklogQuotaImpl) p.backlog_quota_map .getOrDefault(backlogQuotaType, defaultQuota)) .orElse(defaultQuota); @@ -81,16 +78,15 @@ public BacklogQuotaImpl getBacklogQuota(String namespace, String policyPath, Bac } public BacklogQuotaImpl getBacklogQuota(TopicName topicName, BacklogQuotaType backlogQuotaType) { - String policyPath = AdminResource.path(POLICIES, topicName.getNamespace()); if (!isTopicLevelPoliciesEnable) { - return getBacklogQuota(topicName.getNamespace(), policyPath, backlogQuotaType); + return getBacklogQuota(topicName.getNamespaceObject(), backlogQuotaType); } try { return Optional.ofNullable(pulsar.getTopicPoliciesService().getTopicPolicies(topicName)) .map(TopicPolicies::getBackLogQuotaMap) .map(map -> map.get(backlogQuotaType.name())) - .orElseGet(() -> getBacklogQuota(topicName.getNamespace(), policyPath, backlogQuotaType)); + .orElseGet(() -> getBacklogQuota(topicName.getNamespaceObject(), backlogQuotaType)); } catch (BrokerServiceException.TopicPoliciesCacheNotInitException e) { log.debug("Topic policies cache have not init, will apply the namespace backlog quota: topicName={}", topicName); @@ -98,7 +94,7 @@ public BacklogQuotaImpl getBacklogQuota(TopicName topicName, BacklogQuotaType ba log.error("Failed to read topic policies data, " + "will apply the namespace backlog quota: topicName={}", topicName, e); } - return getBacklogQuota(topicName.getNamespace(), policyPath, backlogQuotaType); + return getBacklogQuota(topicName.getNamespaceObject(), backlogQuotaType); } public long getBacklogQuotaLimitInSize(TopicName topicName) { 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 93bffb4d1b821..81c9ee8dd3439 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 @@ -23,9 +23,7 @@ import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun; import static org.apache.commons.collections.CollectionUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.common.events.EventsTopicNames.checkTopicIsEventsNames; -import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -93,16 +91,15 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.cache.BundlesQuotas; -import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerLoader; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl; import org.apache.pulsar.broker.loadbalance.LoadManager; +import org.apache.pulsar.broker.resources.LocalPoliciesResources; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; @@ -118,7 +115,6 @@ import org.apache.pulsar.broker.stats.prometheus.metrics.ObserverGauge; import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; import org.apache.pulsar.broker.systopic.SystemTopicClient; -import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.api.ClientBuilder; @@ -132,7 +128,6 @@ import org.apache.pulsar.common.intercept.BrokerEntryMetadataInterceptor; import org.apache.pulsar.common.intercept.BrokerEntryMetadataUtils; import org.apache.pulsar.common.naming.NamespaceBundle; -import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; @@ -161,9 +156,9 @@ import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.apache.pulsar.common.util.netty.NettyFutureUtil; import org.apache.pulsar.compaction.Compactor; -import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy; import org.slf4j.Logger; @@ -238,11 +233,8 @@ public class BrokerService implements Closeable { private final PulsarStats pulsarStats; private final AuthenticationService authenticationService; - public static final String BROKER_SERVICE_CONFIGURATION_PATH = "/admin/configuration"; public static final String MANAGED_LEDGER_PATH_ZNODE = "/managed-ledgers"; - private final MetadataCache> dynamicConfigurationCache; - private static final LongAdder totalUnackedMessages = new LongAdder(); private final int maxUnackedMessages; public final int maxUnackedMsgsPerDispatcher; @@ -312,9 +304,6 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.backlogQuotaChecker = Executors .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-backlog-quota-checker")); this.authenticationService = new AuthenticationService(pulsar.getConfiguration()); - this.dynamicConfigurationCache = pulsar.getLocalMetadataStore().getMetadataCache( - new TypeReference>() { - }); this.blockedDispatchers = new ConcurrentOpenHashSet<>(); // update dynamic configuration and register-listener updateConfigurationAndRegisterListeners(); @@ -1035,9 +1024,8 @@ public PulsarClient getReplicationClient(String cluster) { return replicationClients.computeIfAbsent(cluster, key -> { try { - String path = PulsarWebResource.path("clusters", cluster); - ClusterData data = pulsar.getPulsarResources().getClusterResources().get(path) - .orElseThrow(() -> new MetadataStoreException.NotFoundException(path)); + ClusterData data = pulsar.getPulsarResources().getClusterResources().getCluster(cluster) + .orElseThrow(() -> new MetadataStoreException.NotFoundException(cluster)); ClientBuilder clientBuilder = PulsarClient.builder() .enableTcpNoDelay(false) .connectionsPerBroker(pulsar.getConfiguration().getReplicationConnectionsPerBroker()) @@ -1110,9 +1098,8 @@ public PulsarAdmin getClusterPulsarAdmin(String cluster) { } return clusterAdmins.computeIfAbsent(cluster, key -> { try { - String path = PulsarWebResource.path("clusters", cluster); - ClusterData data = pulsar.getPulsarResources().getClusterResources().get(path) - .orElseThrow(() -> new MetadataStoreException.NotFoundException(path)); + ClusterData data = pulsar.getPulsarResources().getClusterResources().getCluster(cluster) + .orElseThrow(() -> new MetadataStoreException.NotFoundException(cluster)); ServiceConfiguration conf = pulsar.getConfig(); @@ -1311,8 +1298,9 @@ public CompletableFuture getManagedLedgerConfig(TopicName t ServiceConfiguration serviceConfig = pulsar.getConfiguration(); NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); - return nsr.getPolicies(namespace) - .thenCombine(nsr.getLocalPolicies(namespace), (policies, localPolicies) -> { + LocalPoliciesResources lpr = pulsar.getPulsarResources().getLocalPolicies(); + return nsr.getPoliciesAsync(namespace) + .thenCombine(lpr.getLocalPoliciesAsync(namespace), (policies, localPolicies) -> { PersistencePolicies persistencePolicies = null; RetentionPolicies retentionPolicies = null; OffloadPoliciesImpl topicLevelOffloadPolicies = null; @@ -1758,13 +1746,10 @@ public ConcurrentOpenHashMap>> getTopi private void handleMetadataChanges(Notification n) { - if (n.getPath().startsWith(ConfigurationCacheService.POLICIES_ROOT)) { - Optional ns = NamespaceName.getIfValid( - NamespaceBundleFactory.getNamespaceFromPoliciesPath(n.getPath())); - if (ns.isPresent()) { - handlePoliciesUpdates(ns.get()); - } - } else if (n.getPath().equals(BROKER_SERVICE_CONFIGURATION_PATH)) { + if (n.getType() == NotificationType.Modified && NamespaceResources.pathIsFromNamespace(n.getPath())) { + NamespaceName ns = NamespaceResources.namespaceFromPath(n.getPath()); + handlePoliciesUpdates(ns); + } else if (pulsar().getPulsarResources().getDynamicConfigResources().isDynamicConfigurationPath(n.getPath())) { handleDynamicConfigurationUpdates(); } @@ -1772,7 +1757,7 @@ private void handleMetadataChanges(Notification n) { } private void handlePoliciesUpdates(NamespaceName namespace) { - pulsar.getPulsarResources().getNamespaceResources().getPolicies(namespace) + pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync(namespace) .thenAccept(optPolicies -> { if (!optPolicies.isPresent()) { return; @@ -1802,7 +1787,7 @@ private void handlePoliciesUpdates(NamespaceName namespace) { } private void handleDynamicConfigurationUpdates() { - dynamicConfigurationCache.get(BROKER_SERVICE_CONFIGURATION_PATH) + pulsar().getPulsarResources().getDynamicConfigResources().getDynamicConfigurationAsync() .thenAccept(optMap -> { if (!optMap.isPresent()) { return; @@ -1918,10 +1903,6 @@ public List getAllTopicsFromNamespaceBundle(String namespace, String bund return map2.values(); } - public MetadataCache> getDynamicConfigurationCache() { - return dynamicConfigurationCache; - } - /** * Update dynamic-ServiceConfiguration with value present into zk-configuration-map and register listeners on * dynamic-ServiceConfiguration field to take appropriate action on change of zk-configuration-map. @@ -2146,9 +2127,10 @@ private void validateConfigKey(String key) { private void updateDynamicServiceConfiguration() { Optional> configCache = Optional.empty(); try { - configCache = dynamicConfigurationCache.get(BROKER_SERVICE_CONFIGURATION_PATH).get(); + configCache = + Optional.of(pulsar().getPulsarResources().getDynamicConfigResources().getDynamicConfiguration()); } catch (Exception e) { - log.warn("Failed to read dynamic broker configuration path [{}]:", BROKER_SERVICE_CONFIGURATION_PATH, e); + log.warn("Failed to read dynamic broker configuration", e); } configCache.ifPresent(stringStringMap -> stringStringMap.forEach((key, value) -> { @@ -2277,16 +2259,31 @@ public CompletableFuture fetchPartitionedTopicMetadata .thenCompose(topicExists -> { return fetchPartitionedTopicMetadataAsync(topicName) .thenCompose(metadata -> { - // If topic is already exist, creating partitioned topic is not allowed. - if (metadata.partitions == 0 - && !topicExists - && !topicName.isPartitioned() - && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName) - && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) { - return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName); - } else { - return CompletableFuture.completedFuture(metadata); - } + CompletableFuture future = new CompletableFuture<>(); + + // There are a couple of potentially blocking calls, which we cannot make from the + // MetadataStore callback thread. + pulsar.getExecutor().execute(() -> { + // If topic is already exist, creating partitioned topic is not allowed. + + if (metadata.partitions == 0 + && !topicExists + && !topicName.isPartitioned() + && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName) + && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) { + + pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName) + .thenAccept(md -> future.complete(md)) + .exceptionally(ex -> { + future.completeExceptionally(ex); + return null; + }); + } else { + future.complete(metadata); + } + }); + + return future; }); }); } @@ -2306,7 +2303,7 @@ private CompletableFuture createDefaultPartitionedTopi .thenCompose(__ -> { PartitionedTopicResources partitionResources = pulsar.getPulsarResources().getNamespaceResources() .getPartitionedTopicResources(); - return partitionResources.createAsync(partitionedTopicPath(topicName), configMetadata) + return partitionResources.createPartitionedTopicAsync(topicName, configMetadata) .thenApply(v -> { log.info("partitioned metadata successfully created for {}", topicName); return configMetadata; @@ -2317,20 +2314,12 @@ private CompletableFuture createDefaultPartitionedTopi public CompletableFuture fetchPartitionedTopicMetadataAsync(TopicName topicName) { // gets the number of partitions from the configuration cache return pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() - .getAsync(partitionedTopicPath(topicName)).thenApply(metadata -> { - // if the partitioned topic is not found in zk, then the topic is not partitioned + .getPartitionedTopicMetadataAsync(topicName).thenApply(metadata -> { + // if the partitioned topic is not found in metadata, then the topic is not partitioned return metadata.orElseGet(() -> new PartitionedTopicMetadata()); }); } - private static String partitionedTopicPath(TopicName topicName) { - return String.format("%s/%s/%s/%s", - ConfigurationCacheService.PARTITIONED_TOPICS_ROOT, - topicName.getNamespace(), - topicName.getDomain(), - topicName.getEncodedLocalName()); - } - public OrderedExecutor getTopicOrderedExecutor() { return topicOrderedExecutor; } @@ -2528,8 +2517,8 @@ public int getDefaultNumPartitions(final TopicName topicName) { private AutoTopicCreationOverride getAutoTopicCreationOverride(final TopicName topicName) { try { - Optional policies = pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, topicName.getNamespace())); + Optional policies = + pulsar.getPulsarResources().getNamespaceResources().getPolicies(topicName.getNamespaceObject()); // If namespace policies have the field set, it will override the broker-level setting if (policies.isPresent() && policies.get().autoTopicCreationOverride != null) { return policies.get().autoTopicCreationOverride; @@ -2561,8 +2550,8 @@ public boolean isAllowAutoSubscriptionCreation(final TopicName topicName) { private AutoSubscriptionCreationOverride getAutoSubscriptionCreationOverride(final TopicName topicName) { try { - Optional policies = pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, topicName.getNamespace())); + Optional policies = + pulsar.getPulsarResources().getNamespaceResources().getPolicies(topicName.getNamespaceObject()); // If namespace policies have the field set, it will override the broker-level setting if (policies.isPresent() && policies.get().autoSubscriptionCreationOverride != null) { return policies.get().autoSubscriptionCreationOverride; @@ -2607,7 +2596,7 @@ public CompletableFuture deleteTopicPolicies(TopicName topicName) { private CompletableFuture checkMaxTopicsPerNamespace(TopicName topicName, int numPartitions) { return pulsar.getPulsarResources().getNamespaceResources() - .getPolicies(topicName.getNamespaceObject()) + .getPoliciesAsync(topicName.getNamespaceObject()) .thenCompose(optPolicies -> { int maxTopicsPerNamespace = optPolicies.map(p -> p.max_topics_per_namespace) .orElse(pulsar.getConfig().getMaxTopicsPerNamespace()); 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 97258ec4a636d..472f8a92c2660 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 @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.mledger.impl.EntryCacheManager.create; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import com.carrotsearch.hppc.ObjectObjectHashMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -40,7 +39,6 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -143,7 +141,7 @@ public NonPersistentTopic(String topic, BrokerService brokerService) { public CompletableFuture initialize() { return brokerService.pulsar().getPulsarResources().getNamespaceResources() - .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenAccept(optPolicies -> { if (!optPolicies.isPresent()) { log.warn("[{}] Policies not present and isEncryptionRequired will be set to false", topic); @@ -498,7 +496,7 @@ public CompletableFuture checkReplication() { } return brokerService.pulsar().getPulsarResources().getNamespaceResources() - .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenCompose(optPolicies -> { if (!optPolicies.isPresent()) { return FutureUtil.failedFuture( @@ -903,8 +901,8 @@ public void checkGC() { public void checkInactiveSubscriptions() { TopicName name = TopicName.get(topic); try { - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) + Policies policies = brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPolicies(name.getNamespaceObject()) .orElseThrow(MetadataStoreException.NotFoundException::new); final int defaultExpirationTime = brokerService.pulsar().getConfiguration() .getSubscriptionExpirationTimeMinutes(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index 6aba2c43013e1..0df50cb623f7a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service.persistent; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; import java.util.HashMap; @@ -39,8 +38,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.admin.AdminResource; -import org.apache.pulsar.broker.admin.ZkAdminPaths; import org.apache.pulsar.broker.service.Topic.PublishContext; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.TopicName; @@ -432,8 +429,8 @@ private CompletableFuture isDeduplicationEnabled() { return CompletableFuture.completedFuture(isDeduplicationEnabled.get()); } TopicName name = TopicName.get(topic.getName()); - return pulsar.getConfigurationCache().policiesCache() - .getAsync(AdminResource.path(POLICIES, name.getNamespace())).thenApply(policies -> { + return pulsar.getPulsarResources().getNamespaceResources() + .getPoliciesAsync(name.getNamespaceObject()).thenApply(policies -> { // If namespace policies have the field set, it will override the broker-level setting if (policies.isPresent() && policies.get().deduplicationEnabled != null) { return policies.get().deduplicationEnabled; @@ -495,8 +492,8 @@ public void takeSnapshot() { try { //if topic-level policies not exists, try to get namespace-level policies if (interval == null) { - final Optional policies = pulsar.getConfigurationCache().policiesCache() - .get(ZkAdminPaths.namespacePoliciesPath(TopicName.get(topic.getName()).getNamespaceObject())); + final Optional policies = pulsar.getPulsarResources().getNamespaceResources() + .getPolicies(TopicName.get(topic.getName()).getNamespaceObject()); if (policies.isPresent()) { interval = policies.get().deduplicationSnapshotIntervalSeconds; } 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 2dd55e649f5f7..51e9099a48eb2 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 @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.common.events.EventsTopicNames.checkTopicIsEventsNames; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.carrotsearch.hppc.ObjectObjectHashMap; @@ -75,7 +74,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; @@ -312,7 +310,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS @Override public CompletableFuture initialize() { return brokerService.pulsar().getPulsarResources().getNamespaceResources() - .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenAccept(optPolicies -> { if (!optPolicies.isPresent()) { isEncryptionRequired = false; @@ -607,7 +605,7 @@ private boolean hasRemoteProducers() { public CompletableFuture startReplProducers() { // read repl-cluster from policies to avoid restart of replicator which are in process of disconnect and close return brokerService.pulsar().getPulsarResources().getNamespaceResources() - .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenAccept(optPolicies -> { if (optPolicies.isPresent()) { if (optPolicies.get().replication_clusters != null) { @@ -1329,7 +1327,7 @@ public CompletableFuture checkReplication() { CompletableFuture policiesFuture = brokerService.pulsar().getPulsarResources() .getNamespaceResources() - .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenCompose(optPolicies -> { if (!optPolicies.isPresent()) { return FutureUtil.failedFuture( @@ -1421,7 +1419,7 @@ public CompletableFuture isCompactionEnabled() { TopicName topicName = TopicName.get(topic); return brokerService.getPulsar().getPulsarResources().getNamespaceResources() - .getPolicies(topicName.getNamespaceObject()) + .getPoliciesAsync(topicName.getNamespaceObject()) .thenApply(policies -> { if (policies.isPresent()) { return policies.get().compaction_threshold != null @@ -1441,9 +1439,9 @@ public void checkCompaction() { .map(TopicPolicies::getCompactionThreshold) .orElse(null); if (compactionThreshold == null) { - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(() -> new MetadataStoreException.NotFoundException()); + Policies policies = brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPolicies(name.getNamespaceObject()) + .orElseThrow(() -> new MetadataStoreException.NotFoundException()); compactionThreshold = policies.compaction_threshold; } if (compactionThreshold == null) { @@ -2238,13 +2236,11 @@ private CompletableFuture tryToDeletePartitionedMetadata() { return CompletableFuture.completedFuture(null); } TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); - String path = AdminResource.path(AdminResource.PARTITIONED_TOPIC_PATH_ZNODE, topicName.getNamespace() - , topicName.getDomain().value(), topicName.getEncodedLocalName()); try { PartitionedTopicResources partitionedTopicResources = getBrokerService().pulsar().getPulsarResources() .getNamespaceResources() .getPartitionedTopicResources(); - if (topicName.isPartitioned() && !partitionedTopicResources.exists(path)) { + if (topicName.isPartitioned() && !partitionedTopicResources.partitionedTopicExists(topicName)) { return CompletableFuture.completedFuture(null); } CompletableFuture deleteMetadataFuture = new CompletableFuture<>(); @@ -2269,7 +2265,8 @@ private CompletableFuture tryToDeletePartitionedMetadata() { } } })) - .thenAccept((res) -> partitionedTopicResources.deleteAsync(path).thenAccept((r) -> { + .thenAccept((res) -> partitionedTopicResources.deletePartitionedTopicAsync(topicName) + .thenAccept((r) -> { deleteMetadataFuture.complete(null); }).exceptionally(ex -> { deleteMetadataFuture.completeExceptionally(ex.getCause()); @@ -2292,8 +2289,8 @@ private CompletableFuture tryToDeletePartitionedMetadata() { public void checkInactiveSubscriptions() { TopicName name = TopicName.get(topic); try { - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) + Policies policies = brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPolicies(name.getNamespaceObject()) .orElseThrow(() -> new MetadataStoreException.NotFoundException()); final int defaultExpirationTime = brokerService.pulsar().getConfiguration() .getSubscriptionExpirationTimeMinutes(); @@ -2348,8 +2345,8 @@ private boolean shouldTopicBeRetained() { .orElse(null); if (retentionPolicies == null){ TopicName name = TopicName.get(topic); - retentionPolicies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) + retentionPolicies = brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPolicies(name.getNamespaceObject()) .map(p -> p.retention_policies) .orElse(null); } @@ -2824,7 +2821,7 @@ private CompletableFuture getMessageTTL() { TopicName name = TopicName.get(topic); return brokerService.pulsar().getPulsarResources().getNamespaceResources() - .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenApply(optPolicies -> { if (optPolicies.isPresent()) { if (optPolicies.get().message_ttl_in_seconds != null) { @@ -3212,8 +3209,8 @@ public boolean checkSubscriptionTypesEnable(SubType subType) throws Exception { } private boolean checkNsAndBrokerSubscriptionTypesEnable(TopicName topicName, SubType subType) throws Exception { - Optional policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, topicName.getNamespaceObject().toString())); + Optional policies = brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPolicies(topicName.getNamespaceObject()); if (policies.isPresent()) { if (policies.get().subscription_types_enabled.isEmpty()) { return getBrokerService().getPulsar().getConfiguration() 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 d001262acfc99..16e022d41ee78 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 @@ -21,9 +21,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.pulsar.broker.admin.AdminResource.POLICIES_READONLY_FLAG_PATH; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.RESOURCEGROUPS; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.BoundType; import com.google.common.collect.Lists; @@ -31,9 +28,6 @@ import java.net.MalformedURLException; import java.net.URI; import java.net.URL; -import java.util.ArrayList; -import java.util.Deque; -import java.util.LinkedList; import java.util.List; import java.util.Optional; import java.util.Set; @@ -53,13 +47,11 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; -import org.apache.pulsar.broker.resources.BaseResources; import org.apache.pulsar.broker.resources.BookieResources; import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.resources.DynamicConfigurationResources; @@ -69,6 +61,7 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.ResourceGroupResources; import org.apache.pulsar.broker.resources.TenantResources; +import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; import org.apache.pulsar.common.naming.Constants; @@ -88,8 +81,6 @@ import org.apache.pulsar.common.policies.path.PolicyPath; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.zookeeper.common.PathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -269,7 +260,7 @@ protected static void validateAdminAccessForTenant(PulsarService pulsar, String (isClientAuthenticated(clientAppId)), clientAppId); } - TenantInfo tenantInfo = pulsar.getPulsarResources().getTenantResources().get(path(POLICIES, tenant)) + TenantInfo tenantInfo = pulsar.getPulsarResources().getTenantResources().getTenant(tenant) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Tenant does not exist")); if (pulsar.getConfiguration().isAuthenticationEnabled() && pulsar.getConfiguration().isAuthorizationEnabled()) { @@ -325,7 +316,7 @@ protected static void validateAdminAccessForTenant(PulsarService pulsar, String protected void validateClusterForTenant(String tenant, String cluster) { TenantInfo tenantInfo; try { - tenantInfo = pulsar().getPulsarResources().getTenantResources().get(path(POLICIES, tenant)) + tenantInfo = pulsar().getPulsarResources().getTenantResources().getTenant(tenant) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Tenant does not exist")); } catch (RestException e) { log.warn("Failed to get tenant admin data for tenant {}", tenant); @@ -727,9 +718,9 @@ public static CompletableFuture checkLocalOrGetPeerReplicationC } final CompletableFuture validationFuture = new CompletableFuture<>(); final String localCluster = pulsarService.getConfiguration().getClusterName(); - final String path = AdminResource.path(POLICIES, namespace.toString()); - pulsarService.getConfigurationCache().policiesCache().getAsync(path).thenAccept(policiesResult -> { + pulsarService.getPulsarResources().getNamespaceResources() + .getPoliciesAsync(namespace).thenAccept(policiesResult -> { if (policiesResult.isPresent()) { Policies policies = policiesResult.get(); if (policies.replication_clusters.isEmpty()) { @@ -905,6 +896,10 @@ protected BookieResources bookieResources() { return pulsar().getPulsarResources().getBookieResources(); } + protected TopicResources topicResources() { + return pulsar().getPulsarResources().getTopicResources(); + } + protected NamespaceResources namespaceResources() { return pulsar().getPulsarResources().getNamespaceResources(); } @@ -931,80 +926,23 @@ public static ObjectMapper jsonMapper() { public void validatePoliciesReadOnlyAccess() { try { - if (clusterResources().existsAsync(AdminResource.POLICIES_READONLY_FLAG_PATH).get()) { + if (namespaceResources().getPoliciesReadOnly()) { log.debug("Policies are read-only. Broker cannot do read-write operations"); throw new RestException(Status.FORBIDDEN, "Broker is forbidden to do read-write operations"); } } catch (Exception e) { - log.warn("Unable to fetch read-only policy config {}", POLICIES_READONLY_FLAG_PATH, e); + log.warn("Unable to fetch read-only policy config {}", e); throw new RestException(e); } } protected CompletableFuture hasActiveNamespace(String tenant) { - CompletableFuture activeNamespaceFuture = new CompletableFuture<>(); - tenantResources().getChildrenAsync(path(POLICIES, tenant)).thenAccept(clusterOrNamespaceList -> { - if (clusterOrNamespaceList == null || clusterOrNamespaceList.isEmpty()) { - activeNamespaceFuture.complete(null); - return; - } - List> activeNamespaceListFuture = Lists.newArrayList(); - clusterOrNamespaceList.forEach(clusterOrNamespace -> { - // get list of active V1 namespace - CompletableFuture checkNs = new CompletableFuture<>(); - activeNamespaceListFuture.add(checkNs); - tenantResources().getChildrenAsync(path(POLICIES, tenant, clusterOrNamespace)) - .whenComplete((children, ex) -> { - if (ex != null) { - checkNs.completeExceptionally(ex); - return; - } - if (children != null && !children.isEmpty()) { - checkNs.completeExceptionally( - new RestException(Status.PRECONDITION_FAILED, "Tenant has active namespace")); - return; - } - String namespace = NamespaceName.get(tenant, clusterOrNamespace).toString(); - // if the length is 0 then this is probably a leftover cluster from namespace - // created - // with the v1 admin format (prop/cluster/ns) and then deleted, so no need to - // add it to the list - namespaceResources().getAsync(path(POLICIES, namespace)).thenApply(data -> { - if (data.isPresent()) { - checkNs.completeExceptionally(new RestException(Status.PRECONDITION_FAILED, - "Tenant has active namespace")); - } else { - checkNs.complete(null); - } - return null; - }).exceptionally(ex2 -> { - if (ex2.getCause() instanceof MetadataStoreException.ContentDeserializationException) { - // it's not a valid namespace-node - checkNs.complete(null); - } else { - checkNs.completeExceptionally( - new RestException(Status.INTERNAL_SERVER_ERROR, ex2.getCause())); - } - return null; - }); - }); - FutureUtil.waitForAll(activeNamespaceListFuture).thenAccept(r -> { - activeNamespaceFuture.complete(null); - }).exceptionally(ex -> { - activeNamespaceFuture.completeExceptionally(ex.getCause()); - return null; - }); - }); - }).exceptionally(ex -> { - activeNamespaceFuture.completeExceptionally(ex.getCause()); - return null; - }); - return activeNamespaceFuture; + return tenantResources().hasActiveNamespace(tenant); } protected void validateClusterExists(String cluster) { try { - if (!clusterResources().get(path("clusters", cluster)).isPresent()) { + if (!clusterResources().clusterExists(cluster)) { throw new RestException(Status.PRECONDITION_FAILED, "Cluster " + cluster + " does not exist."); } } catch (Exception e) { @@ -1021,7 +959,7 @@ protected CompletableFuture canUpdateCluster(String tenant, Set ol } CompletableFuture checkNs = new CompletableFuture<>(); activeNamespaceFuture.add(checkNs); - tenantResources().getChildrenAsync(path(POLICIES, tenant, cluster)).whenComplete((activeNamespaces, ex) -> { + tenantResources().getActiveNamespaces(tenant, cluster).whenComplete((activeNamespaces, ex) -> { if (ex != null) { log.warn("Failed to get namespaces under {}-{}, {}", tenant, cluster, ex.getCause().getMessage()); checkNs.completeExceptionally(ex.getCause()); @@ -1035,8 +973,7 @@ protected CompletableFuture canUpdateCluster(String tenant, Set ol checkNs.complete(null); }); } - return activeNamespaceFuture.isEmpty() ? CompletableFuture.completedFuture(null) - : FutureUtil.waitForAll(activeNamespaceFuture); + return FutureUtil.waitForAll(activeNamespaceFuture); } /** @@ -1105,92 +1042,4 @@ && pulsar().getBrokerService().isAuthorizationEnabled()) { } } - /* - * Get the list of namespaces (on every cluster) for a given property. - * - * @param property the property name - * @return the list of namespaces - */ - protected List getListOfNamespaces(String tenant) throws Exception { - List namespaces = Lists.newArrayList(); - - if (!tenantResources().exists(path(POLICIES, tenant))) { - throw new RestException(Status.NOT_FOUND, tenant + " doesn't exist"); - } - // this will return a cluster in v1 and a namespace in v2 - for (String clusterOrNamespace : tenantResources().getChildren(path(POLICIES, tenant))) { - // Then get the list of namespaces - final List children = tenantResources().getChildren(path(POLICIES, tenant, clusterOrNamespace)); - if (children == null || children.isEmpty()) { - String namespace = NamespaceName.get(tenant, clusterOrNamespace).toString(); - // if the length is 0 then this is probably a leftover cluster from namespace created - // with the v1 admin format (prop/cluster/ns) and then deleted, so no need to add it to the list - try { - if (namespaceResources().get(path(POLICIES, namespace)).isPresent()) { - namespaces.add(namespace); - } - } catch (MetadataStoreException.ContentDeserializationException e) { - // not a namespace node - } - - } else { - children.forEach(ns -> { - namespaces.add(NamespaceName.get(tenant, clusterOrNamespace, ns).toString()); - }); - } - } - - namespaces.sort(null); - return namespaces; - } - - /** - * Get the list of resourcegroups. - * - * @return the list of resourcegroups - */ - - protected List getListOfResourcegroups(String property) throws Exception { - List resourcegroups = Lists.newArrayList(); - - for (String resourcegroup : resourceGroupResources().getChildren(path(RESOURCEGROUPS))) { - resourcegroups.add(resourcegroup); - } - - resourcegroups.sort(null); - return resourcegroups; - } - - - public static void deleteRecursive(BaseResources resources, final String pathRoot) throws MetadataStoreException { - PathUtils.validatePath(pathRoot); - List tree = listSubTreeBFS(resources, pathRoot); - log.debug("Deleting {} with size {}", tree, tree.size()); - log.debug("Deleting " + tree.size() + " subnodes "); - for (int i = tree.size() - 1; i >= 0; --i) { - // Delete the leaves first and eventually get rid of the root - resources.delete(tree.get(i)); - } - } - - public static List listSubTreeBFS(BaseResources resources, final String pathRoot) - throws MetadataStoreException { - Deque queue = new LinkedList<>(); - List tree = new ArrayList<>(); - queue.add(pathRoot); - tree.add(pathRoot); - while (true) { - String node = queue.pollFirst(); - if (node == null) { - break; - } - List children = resources.getChildren(node); - for (final String child : children) { - final String childPath = node + "/" + child; - queue.add(childPath); - tree.add(childPath); - } - } - return tree; - } } 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 21ff33dfcbeb3..7e1c576170636 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 @@ -71,7 +71,6 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; -import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -579,10 +578,10 @@ public void testUpdateDynamicConfigurationWithZkWatch() throws Exception { * @throws Exception */ @Test - public void testInvalidDynamicConfigContentInZK() throws Exception { + public void testInvalidDynamicConfigContentInMetadata() throws Exception { final int newValue = 10; - // set invalid data into dynamic-config znode so, broker startup fail to deserialize data - pulsar.getLocalMetadataStore().put(BrokerService.BROKER_SERVICE_CONFIGURATION_PATH, "$".getBytes(), + // set invalid data into dynamic-config node so, broker startup fail to deserialize data + pulsar.getLocalMetadataStore().put("/admin/configuration", "$".getBytes(), Optional.empty()).join(); stopBroker(); @@ -593,7 +592,7 @@ public void testInvalidDynamicConfigContentInZK() throws Exception { Map configMap = Maps.newHashMap(); configMap.put("brokerShutdownTimeoutMs", Integer.toString(newValue)); - pulsar.getLocalMetadataStore().put(BrokerService.BROKER_SERVICE_CONFIGURATION_PATH, + pulsar.getLocalMetadataStore().put("/admin/configuration", ObjectMapperFactory.getThreadLocal().writeValueAsBytes(configMap), Optional.empty()).join(); // wait config to be updated 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 0180a7e69aee7..06d066f4c38b2 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 @@ -502,7 +502,7 @@ public void testInvalidDynamicConfigContentInZK() throws Exception { final int newValue = 10; // set invalid data into dynamic-config znode so, broker startup fail to deserialize data - pulsar.getLocalMetadataStore().put(BrokerService.BROKER_SERVICE_CONFIGURATION_PATH, "$".getBytes(), + pulsar.getLocalMetadataStore().put("/admin/configuration", "$".getBytes(), Optional.empty()).join(); stopBroker(); @@ -512,7 +512,7 @@ public void testInvalidDynamicConfigContentInZK() throws Exception { // update zk with config-value which should fire watch and broker should update the config value Map configMap = Maps.newHashMap(); configMap.put("brokerShutdownTimeoutMs", Integer.toString(newValue)); - pulsar.getLocalMetadataStore().put(BrokerService.BROKER_SERVICE_CONFIGURATION_PATH, + pulsar.getLocalMetadataStore().put("/admin/configuration", ObjectMapperFactory.getThreadLocal().writeValueAsBytes(configMap), Optional.empty()).join(); // wait config to be updated 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 f49a3e3e60ffe..5a6b7845fe7f4 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 @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.loadbalance; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -26,7 +25,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; - import com.google.common.collect.Lists; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -47,18 +45,16 @@ import org.apache.bookkeeper.util.ZkUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.loadbalance.impl.PulsarResourceDescription; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.internal.NamespacesImpl; +import org.apache.pulsar.common.naming.NamespaceName; 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; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; @@ -578,8 +574,7 @@ private BundlesData getBundles(int numBundles) { private void createNamespace(PulsarService pulsar, String namespace, int numBundles) throws Exception { Policies policies = new Policies(); policies.bundles = getBundles(numBundles); - String path = AdminResource.path(POLICIES, namespace); - pulsar.getPulsarResources().getNamespaceResources().create(path, policies); + pulsar.getPulsarResources().getNamespaceResources().createPolicies(NamespaceName.get(namespace), policies); } @@ -768,13 +763,12 @@ private void createNamespacePolicies(PulsarService pulsar) throws Exception { .build(); policies.setPolicy("otherBrokerPolicy", policyData); - String path = AdminResource.path("clusters", "use", "namespaceIsolationPolicies"); try { - pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().create(path, + pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().createIsolationData("use", policies.getPolicies()); } catch (BadVersionException e) { // isolation policy already exist - pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().set(path, + pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().setIsolationData("use", data -> policies.getPolicies()); } } 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 34bf5565ab8ce..163089a40ae96 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 @@ -48,7 +48,6 @@ import org.apache.commons.lang3.SystemUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.loadbalance.impl.GenericBrokerHostUsageImpl; import org.apache.pulsar.broker.loadbalance.impl.LinuxBrokerHostUsageImpl; import org.apache.pulsar.broker.loadbalance.impl.PulsarLoadReportImpl; @@ -181,13 +180,12 @@ private void createNamespacePolicies(PulsarService pulsar) throws Exception { .build(); policies.setPolicy("primaryBrokerPolicy", policyData); - String path = AdminResource.path("clusters", "use", "namespaceIsolationPolicies"); try { - pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().create(path, + pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().createIsolationData("use", policies.getPolicies()); } catch (BadVersionException e) { // isolation policy already exist - pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().set(path, + pulsar.getPulsarResources().getNamespaceResources().getIsolationPolicies().setIsolationData("use", data -> policies.getPolicies()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java index cded717391e61..91260098a3c31 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java @@ -20,6 +20,7 @@ import com.google.common.collect.Sets; import io.prometheus.client.Summary; +import java.util.Collections; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; import org.apache.pulsar.broker.resourcegroup.ResourceGroupService.ResourceGroupUsageStatsType; @@ -32,6 +33,7 @@ import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +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.TenantInfo; @@ -103,8 +105,8 @@ public void testMTProduceConsumeRGUsage() throws Exception { // NonPersistentTopicNamesDifferentTenantAndNsRGs[]. // If the number of RGs is == 1, the "same tenant/NS" topics names and the // "different tenant/NS" topic names coincide; so, there's no need to run them distinctly. - if (NumRGs == 1 && (topicSet % 2 == 1)) { - log.info("NumRGs={}; skipping repetition of test for topic-set {}", NumRGs, topicSet); + if (NUM_RESOURCE_GROUPS == 1 && (topicSet % 2 == 1)) { + log.info("NumRGs={}; skipping repetition of test for topic-set {}", NUM_RESOURCE_GROUPS, topicSet); } else { testProduceConsumeUsageOnRG(topicStrings); log.info("Done with topic-set {}", topicSet); @@ -129,7 +131,7 @@ private class produceMessages implements Runnable { producerId = prodId; numMesgsToProduce = nMesgs; topicStrings = topics; - myProduceTopic = topicStrings[producerId % NumTopics]; + myProduceTopic = topicStrings[producerId % NUM_TOPICS]; } public int getNumBytesSent() { @@ -209,7 +211,7 @@ private class consumeMessages implements Runnable { private int recvdNumMsgs = 0; private int numExceptions = 0; private volatile boolean allMessagesReceived = false; - private boolean consumerIsReady = false; + private volatile boolean consumerIsReady = false; consumeMessages(int consId, int nMesgs, int totalMesgs, SubscriptionType subType, String[] topics) { consumerId = consId; @@ -239,6 +241,7 @@ public int getNumExceptions() { public void closeConsumer() { try { + consumerIsReady = false; consumer.close(); } catch (PulsarClientException p) { numExceptions++; @@ -287,7 +290,7 @@ public void run() { Message message = null; consumerIsReady = true; - while (!allMessagesReceived) { + while (consumerIsReady && !allMessagesReceived) { log.debug("Consumer={} waiting for mesgnum={}", consumerId, recvdNumMsgs); try { message = consumer.receive(recvTimeoutMilliSecs, TimeUnit.MILLISECONDS); @@ -361,7 +364,7 @@ private void registerTenantsAndNamespaces(String[] topicStrings) throws Exceptio final TopicName topic = TopicName.get(topicStr); final String tenantRG = TopicToTenantRGName(topic); final String namespaceRG = TopicToNamespaceRGName(topic); - final String tenantAndNamespace = topic.getNamespace(); + final NamespaceName ns = topic.getNamespaceObject(); // The tenant name and namespace name parts of the topic are the same as their corresponding RG-names. // Hence, the arguments to register look a little odd. @@ -370,7 +373,7 @@ private void registerTenantsAndNamespaces(String[] topicStrings) throws Exceptio registeredTenants.add(tenantRG); } if (!registeredNamespaces.contains(namespaceRG)) { - this.rgservice.registerNameSpace(namespaceRG, tenantAndNamespace); + this.rgservice.registerNameSpace(namespaceRG, ns); registeredNamespaces.add(namespaceRG); } } @@ -390,7 +393,7 @@ private void unRegisterTenantsAndNamespaces(String[] topicStrings) throws Except registeredTenants.remove(tenantRG); } if (registeredNamespaces.contains(namespaceRG)) { - this.rgservice.unRegisterNameSpace(namespaceRG, tenantAndNamespace); + this.rgservice.unRegisterNameSpace(namespaceRG, NamespaceName.get(tenantAndNamespace)); registeredNamespaces.remove(namespaceRG); } } @@ -404,13 +407,13 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception // createTopics(topicStrings); registerTenantsAndNamespaces(topicStrings); - final int TotalExpectedMessagesToSend = NumTotalMessages; + final int TotalExpectedMessagesToSend = NUM_TOTAL_MESSAGES; final int TotalExpectedMessagesToReceive = TotalExpectedMessagesToSend; final SubscriptionType consumeSubscriptionType = SubscriptionType.Shared; // Shared, or Exclusive - producerWithThread prodThr[] = new producerWithThread[NumProducers]; - consumerWithThread consThr[] = new consumerWithThread[NumConsumers]; + producerWithThread prodThr[] = new producerWithThread[NUM_PRODUCERS]; + consumerWithThread consThr[] = new consumerWithThread[NUM_CONSUMERS]; int sentNumBytes = 0; int sentNumMsgs = 0; int numProducerExceptions = 0; @@ -419,9 +422,9 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception // this.verfyRGProdConsStats(topicStrings, 0, 0, 0, 0,true, true); // Fork some consumers to receive the messages. - for (int ix = 0; ix < NumConsumers; ix++) { + for (int ix = 0; ix < NUM_CONSUMERS; ix++) { consThr[ix] = new consumerWithThread(); - consumeMessages cm = new consumeMessages(ix, NumMessagesPerConsumer, TotalExpectedMessagesToReceive, + consumeMessages cm = new consumeMessages(ix, NUM_MESSAGES_PER_CONSUMER, TotalExpectedMessagesToReceive, consumeSubscriptionType, topicStrings); Thread thr = new Thread(cm); thr.start(); @@ -434,18 +437,18 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception do { Thread.sleep(500); numReadyConsumers = 0; - for (int ix = 0; ix < NumConsumers; ix++) { + for (int ix = 0; ix < NUM_CONSUMERS; ix++) { if (consThr[ix].consumer.isConsumerReady()) { numReadyConsumers++; } } - log.debug("{} consumers are not yet ready", NumConsumers - numReadyConsumers); - } while (numReadyConsumers < NumConsumers); + log.debug("{} consumers are not yet ready", NUM_CONSUMERS - numReadyConsumers); + } while (numReadyConsumers < NUM_CONSUMERS); // Fork some producers to send the messages. - for (int ix = 0; ix < NumProducers; ix++) { + for (int ix = 0; ix < NUM_PRODUCERS; ix++) { prodThr[ix] = new producerWithThread(); - produceMessages pm = new produceMessages(ix, NumMessagesPerProducer, topicStrings); + produceMessages pm = new produceMessages(ix, NUM_MESSAGES_PER_PRODUCER, topicStrings); Thread thr = new Thread(pm); thr.start(); prodThr[ix].producer = pm; @@ -454,7 +457,7 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception // Wait for the producers to complete. int sentMsgs, sentBytes; - for (int ix = 0; ix < NumProducers; ix++) { + for (int ix = 0; ix < NUM_PRODUCERS; ix++) { prodThr[ix].thread.join(); sentBytes = prodThr[ix].producer.getNumBytesSent(); sentMsgs = prodThr[ix].producer.getNumMessagesSent(); @@ -476,7 +479,7 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception Thread.sleep(2000); recvdNumMsgs = 0; int consNumMesgsRecvd; - for (int ix = 0; ix < NumConsumers; ix++) { + for (int ix = 0; ix < NUM_CONSUMERS; ix++) { consNumMesgsRecvd = consThr[ix].consumer.getNumMessagesRecvd(); recvdNumMsgs += consNumMesgsRecvd; log.debug("consumer={} received {} messages (current total {}, expected {})", @@ -485,12 +488,12 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception } while (recvdNumMsgs < TotalExpectedMessagesToReceive); // Tell the consumers that all expected messages have been received (but don't close them yet). - for (int ix = 0; ix < NumConsumers; ix++) { + for (int ix = 0; ix < NUM_CONSUMERS; ix++) { consThr[ix].consumer.setAllMessagesReceived(); log.debug("consumer={} told to stop", ix); } - boolean[] joinedConsumers = new boolean[NumConsumers]; + boolean[] joinedConsumers = new boolean[NUM_CONSUMERS]; for (boolean b : joinedConsumers) { b = false; } @@ -498,8 +501,8 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception recvdNumMsgs = recvdNumBytes = 0; int numConsumersDone = 0; int recvdMsgs, recvdBytes; - while (numConsumersDone < NumConsumers) { - for (int ix = 0; ix < NumConsumers; ix++) { + while (numConsumersDone < NUM_CONSUMERS) { + for (int ix = 0; ix < NUM_CONSUMERS; ix++) { if (joinedConsumers[ix] == false) { recvdBytes = consThr[ix].consumer.getNumBytesRecvd(); recvdMsgs = consThr[ix].consumer.getNumMessagesRecvd(); @@ -518,7 +521,7 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception } // Close the consumers. - for (int ix = 0; ix < NumConsumers; ix++) { + for (int ix = 0; ix < NUM_CONSUMERS; ix++) { consThr[ix].consumer.closeConsumer(); } @@ -573,7 +576,7 @@ private void verfyRGProdConsStats(String[] topicStrings, // tenant RG, and again on the namespace RG. This double-counting is avoided if tenant-RG == NS-RG. // This is a known (and discussed) artifact in the implementation. // 'ScaleFactor' is a way to incorporate that effect in the verification. - final int ScaleFactor = tenantRGEqualsNsRG ? 1 : 2; + final int ScaleFactor = 1; // tenantRGEqualsNsRG ? 1 : 2; // Following sleep is to get the broker-service to gather stats on the topics. // [There appears to be some asynchrony there.] @@ -659,14 +662,14 @@ private void verfyRGProdConsStats(String[] topicStrings, if (checkProduce) { prodCounts = ResourceGroup.accumulateBMCount(totalTenantRGProdCounts, totalNsRGProdCounts); - Assert.assertEquals(prodCounts.messages, sentNumMsgs * ScaleFactor); - Assert.assertTrue(prodCounts.bytes >= ExpectedNumBytesSent * ScaleFactor); + Assert.assertEquals(prodCounts.messages, sentNumMsgs); + Assert.assertTrue(prodCounts.bytes >= ExpectedNumBytesSent); } if (checkConsume) { consCounts = ResourceGroup.accumulateBMCount(totalTenantRGConsCounts, totalNsRGConsCounts); - Assert.assertEquals(consCounts.messages, recvdNumMsgs * ScaleFactor); - Assert.assertTrue(consCounts.bytes >= ExpectedNumBytesReceived * ScaleFactor); + Assert.assertEquals(consCounts.messages, recvdNumMsgs); + Assert.assertTrue(consCounts.bytes >= ExpectedNumBytesReceived); } } @@ -690,12 +693,6 @@ private void verifyRGMetrics(String[] topicStrings, long[] totalUsageReportCounts = new long[ResourceGroupMonitoringClass.values().length]; long totalUpdates = 0; - // If the tenant and NS are on different RGs, the bytes/messages get counted once on the - // tenant RG, and again on the namespace RG. This double-counting is avoided if tenant-RG == NS-RG. - // This is a known (and discussed) artifact in the implementation. - // 'ScaleFactor' is a way to incorporate that effect in the verification. - final int ScaleFactor = tenantRGEqualsNsRG ? 1 : 2; - // Hack to ensure aggregator calculation without waiting for a period of aggregation. // [aggregateResourceGroupLocalUsages() is idempotent when there's no new traffic flowing.] this.rgservice.aggregateResourceGroupLocalUsages(); @@ -728,8 +725,8 @@ private void verifyRGMetrics(String[] topicStrings, totalTenantRegisters, totalTenantUnRegisters, totalNamespaceRegisters, totalNamespaceUnRegisters); // On each run, there will be 'NumRGs' registrations - Assert.assertEquals(totalTenantRegisters - residualTenantRegs, NumRGs); - Assert.assertEquals(totalNamespaceRegisters - residualNamespaceRegs, NumRGs); + Assert.assertEquals(totalTenantRegisters - residualTenantRegs, NUM_RESOURCE_GROUPS); + Assert.assertEquals(totalNamespaceRegisters - residualNamespaceRegs, NUM_RESOURCE_GROUPS); // The unregisters will lag the registers by one round (because verifyRGMetrics() is called // prior to unregister). In other words, their numbers will equal the residuals for the registers. @@ -751,14 +748,14 @@ private void verifyRGMetrics(String[] topicStrings, // So, we take the residuals into account when comparing against the expected. if (checkProduce && mc == ResourceGroupMonitoringClass.Publish) { Assert.assertEquals(totalUsedMessages[mcIdx] - residualSentNumMessages, - sentNumMsgs * ScaleFactor); + sentNumMsgs); Assert.assertTrue(totalUsedBytes[mcIdx] - residualSentNumBytes - >= ExpectedNumBytesSent * ScaleFactor); + >= ExpectedNumBytesSent); } else if (checkConsume && mc == ResourceGroupMonitoringClass.Dispatch) { Assert.assertEquals(totalUsedMessages[mcIdx] - residualRecvdNumMessages, - recvdNumMsgs * ScaleFactor); + recvdNumMsgs); Assert.assertTrue(totalUsedBytes[mcIdx] - residualRecvdNumBytes - >= ExpectedNumBytesReceived * ScaleFactor); + >= ExpectedNumBytesReceived); } long perClassUsageReports = numLocalUsageReports / ResourceGroupMonitoringClass.values().length; @@ -766,10 +763,10 @@ private void verifyRGMetrics(String[] topicStrings, } // Update the residuals for next round of tests. - residualSentNumBytes += sentNumBytes * ScaleFactor; - residualSentNumMessages += sentNumMsgs * ScaleFactor; - residualRecvdNumBytes += recvdNumBytes * ScaleFactor; - residualRecvdNumMessages += recvdNumMsgs * ScaleFactor; + residualSentNumBytes += sentNumBytes; + residualSentNumMessages += sentNumMsgs; + residualRecvdNumBytes += recvdNumBytes; + residualRecvdNumMessages += recvdNumMsgs; Assert.assertEquals(totalUpdates, 0); // currently, we don't update the RGs in this UT @@ -794,26 +791,26 @@ private void verifyRGMetrics(String[] topicStrings, private static final Logger log = LoggerFactory.getLogger(RGUsageMTAggrWaitForAllMesgsTest.class); // Empirically, there appears to be a 45-byte overhead for metadata, imposed by Pulsar runtime. - private final int PER_MESSAGE_METADATA_OHEAD = 45; + private static final int PER_MESSAGE_METADATA_OHEAD = 45; private static final int PUBLISH_INTERVAL_SECS = 10; - private final int NumProducers = 32; - private final int NumConsumers = 64; - private final int NumMessagesPerProducer = 100; - private final int NumTopics = 32; // Set == NumProducers, so each producer can send on its own topic - private final int NumRGs = 16; // arbitrarily, half of NumTopics, so 2 topics map to each RG - private final int NumTotalMessages = NumMessagesPerProducer * NumProducers; - private final int NumMessagesPerConsumer = NumTotalMessages / NumConsumers; + private static final int NUM_PRODUCERS = 4; + private static final int NUM_CONSUMERS = 4; + private static final int NUM_MESSAGES_PER_PRODUCER = 100; + private static final int NUM_TOPICS = 8; // Set == NumProducers, so each producer can send on its own topic + private static final int NUM_RESOURCE_GROUPS = 4; // arbitrarily, half of NumTopics, so 2 topics map to each RG + private static final int NUM_TOTAL_MESSAGES = NUM_MESSAGES_PER_PRODUCER * NUM_PRODUCERS; + private static final int NUM_MESSAGES_PER_CONSUMER = NUM_TOTAL_MESSAGES / NUM_CONSUMERS; private final org.apache.pulsar.common.policies.data.ResourceGroup rgConfig = new org.apache.pulsar.common.policies.data.ResourceGroup(); private ResourceGroupService rgservice; - private ResourceGroup[] resGroups = new ResourceGroup[NumRGs]; + private ResourceGroup[] resGroups = new ResourceGroup[NUM_RESOURCE_GROUPS]; private final String clusterName = "test"; private final String BaseRGName = "rg-"; private final String BaseTestTopicName = "rgusage-topic-"; - private String[] RGNames = new String[NumRGs]; + private String[] RGNames = new String[NUM_RESOURCE_GROUPS]; // The number of times we pretend to have not suppressed sending a local usage report. private long numLocalUsageReports; @@ -823,28 +820,31 @@ private void verifyRGMetrics(String[] topicStrings, // [This is required to test the special case of "tenant and NS refer to the same RG", because in that case // we don't double-count the usage.] // Same-order mapping: e.g., rg-0/rg-0 (for 0th entry) - private String[] TenantAndNsNameSameOrder = new String[NumRGs]; + private String[] TenantAndNsNameSameOrder = new String[NUM_RESOURCE_GROUPS]; // Opposite order mapping: e.g., rg-0/rg-49 (for 0th entry with 50 RGs) - private String[] TenantAndNsNameOppositeOrder = new String[NumRGs]; + private String[] TenantAndNsNameOppositeOrder = new String[NUM_RESOURCE_GROUPS]; // Similar to above (same and opposite order) for topics. // E.g., rg-0/rg-0/rgusage-topic0 for 0-th topic in "same order" // and rg-0/rg-49/rgusage-topic0 for 0-th topic in "opposite order", with 50 RGs - private String[] TopicNamesSameTenantAndNsRGs = new String[NumTopics]; - private String[] TopicNamesDifferentTenantAndNsRGs = new String[NumTopics]; + private String[] TopicNamesSameTenantAndNsRGs = new String[NUM_TOPICS]; + private String[] TopicNamesDifferentTenantAndNsRGs = new String[NUM_TOPICS]; // Persistent and non-persistent topic strings with the above names. - private String[] PersistentTopicNamesSameTenanatAndNsRGs = new String[NumTopics]; - private String[] PersistentTopicNamesDifferentTenantAndNsRGs = new String[NumTopics]; - private String[] NonPersistentTopicNamesSameTenantAndNsRGs = new String[NumTopics]; - private String[] NonPersistentTopicNamesDifferentTenantAndNsRGs = new String[NumTopics]; + private String[] PersistentTopicNamesSameTenanatAndNsRGs = new String[NUM_TOPICS]; + private String[] PersistentTopicNamesDifferentTenantAndNsRGs = new String[NUM_TOPICS]; + private String[] NonPersistentTopicNamesSameTenantAndNsRGs = new String[NUM_TOPICS]; + private String[] NonPersistentTopicNamesDifferentTenantAndNsRGs = new String[NUM_TOPICS]; private List AllTopicNames = Arrays.asList( PersistentTopicNamesSameTenanatAndNsRGs, PersistentTopicNamesDifferentTenantAndNsRGs, NonPersistentTopicNamesSameTenantAndNsRGs, - NonPersistentTopicNamesDifferentTenantAndNsRGs); + NonPersistentTopicNamesDifferentTenantAndNsRGs + ); + + // We don't periodically report to a remote broker in this test. So, we will use cumulative stats. private final ResourceGroupUsageStatsType getCumulativeUsageStats = ResourceGroupUsageStatsType.Cumulative; @@ -919,18 +919,18 @@ private void prepareRGs() throws Exception { // Check for a few invariants which allow easier mapping of structures in the test. // Ensure that the number of consumers is a multiple of the number of producers. - Assert.assertTrue(NumConsumers >= NumProducers && NumConsumers % NumProducers == 0); + Assert.assertTrue(NUM_CONSUMERS >= NUM_PRODUCERS && NUM_CONSUMERS % NUM_PRODUCERS == 0); // Number of messages is a multiple of the number of topics. - Assert.assertEquals(NumTotalMessages % NumTopics, 0); + Assert.assertEquals(NUM_TOTAL_MESSAGES % NUM_TOPICS, 0); // Ensure that the number of topics is a multiple of the number of RGs. - Assert.assertEquals(NumTopics % NumRGs, 0); + Assert.assertEquals(NUM_TOPICS % NUM_RESOURCE_GROUPS, 0); // Ensure that the messages-per-consumer is an integral multiple of the number of consumers. - final int NumConsumerMessages = NumMessagesPerConsumer * NumConsumers; - final int NumProducerMessages = NumMessagesPerProducer * NumProducers; - Assert.assertTrue(NumMessagesPerConsumer > 0 && NumConsumerMessages == NumProducerMessages); + final int NumConsumerMessages = NUM_MESSAGES_PER_CONSUMER * NUM_CONSUMERS; + final int NumProducerMessages = NUM_MESSAGES_PER_PRODUCER * NUM_PRODUCERS; + Assert.assertTrue(NUM_MESSAGES_PER_CONSUMER > 0 && NumConsumerMessages == NumProducerMessages); rgConfig.setPublishRateInBytes(1500); rgConfig.setPublishRateInMsgs(100); @@ -938,25 +938,25 @@ private void prepareRGs() throws Exception { rgConfig.setDispatchRateInMsgs(500); // Set up the RG names; creation of RGs will be done elsewhere. - for (int ix = 0; ix < NumRGs; ix++) { + for (int ix = 0; ix < NUM_RESOURCE_GROUPS; ix++) { RGNames[ix] = BaseRGName + ix; } // Create all the tenants final TenantInfo configInfo = new TenantInfoImpl(Sets.newHashSet("fakeAdminRole"), Sets.newHashSet(clusterName)); - for (int ix = 0; ix < NumRGs; ix++) { + for (int ix = 0; ix < NUM_RESOURCE_GROUPS; ix++) { admin.tenants().createTenant(RGNames[ix], configInfo); } // Set up the tenant-and-nsname mapping strings, for same and opposite order of RGs. - for (int ix = 0; ix < NumRGs; ix++) { + for (int ix = 0; ix < NUM_RESOURCE_GROUPS; ix++) { TenantAndNsNameSameOrder[ix] = RGNames[ix] + "/" + RGNames[ix]; - TenantAndNsNameOppositeOrder[ix] = RGNames[ix] + "/" + RGNames[NumRGs - (ix+1)]; + TenantAndNsNameOppositeOrder[ix] = RGNames[ix] + "/" + RGNames[NUM_RESOURCE_GROUPS - (ix+1)]; } // Create all the namespaces - for (int ix = 0; ix < NumRGs; ix++) { + for (int ix = 0; ix < NUM_RESOURCE_GROUPS; ix++) { if (!createdNamespaces.contains(TenantAndNsNameSameOrder[ix])) { admin.namespaces().createNamespace(TenantAndNsNameSameOrder[ix]); admin.namespaces().setNamespaceReplicationClusters( @@ -973,19 +973,20 @@ private void prepareRGs() throws Exception { } // Create all the topic name strings - for (int ix = 0; ix < NumTopics; ix++) { + for (int ix = 0; ix < NUM_TOPICS; ix++) { TopicNamesSameTenantAndNsRGs[ix] = - TenantAndNsNameSameOrder[ix % NumRGs] + "/" + BaseTestTopicName + ix; + TenantAndNsNameSameOrder[ix % NUM_RESOURCE_GROUPS] + "/" + BaseTestTopicName + ix; TopicNamesDifferentTenantAndNsRGs[ix] = - TenantAndNsNameOppositeOrder[ix % NumRGs] + "/" + BaseTestTopicName + ix; + TenantAndNsNameOppositeOrder[ix % NUM_RESOURCE_GROUPS] + "/" + BaseTestTopicName + ix; } // Create all the persistent and non-persistent topic strings - for (int ix = 0; ix < NumTopics; ix++) { + for (int ix = 0; ix < NUM_TOPICS; ix++) { PersistentTopicNamesSameTenanatAndNsRGs[ix] = "persistent://" + TopicNamesSameTenantAndNsRGs[ix]; PersistentTopicNamesDifferentTenantAndNsRGs[ix] = "persistent://" + TopicNamesDifferentTenantAndNsRGs[ix]; + NonPersistentTopicNamesSameTenantAndNsRGs[ix] = "non-persistent://" + TopicNamesSameTenantAndNsRGs[ix]; NonPersistentTopicNamesDifferentTenantAndNsRGs[ix] = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java index c151f25ad38c0..8a85d4f90fd39 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java @@ -18,18 +18,17 @@ */ package org.apache.pulsar.broker.resourcegroup; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.RESOURCEGROUPS; -import static org.apache.pulsar.common.policies.path.PolicyPath.path; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import com.google.common.collect.Sets; import java.util.Random; - import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.resources.ResourceGroupResources; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ResourceGroup; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -126,11 +125,13 @@ public void testResourceGroupAttachToNamespace() throws Exception { admin.namespaces().setNamespaceResourceGroup(namespaceName, rgName); Awaitility.await().untilAsserted(() -> - assertNotNull(pulsar.getResourceGroupServiceManager().getNamespaceResourceGroup(namespaceName))); + assertNotNull(pulsar.getResourceGroupServiceManager() + .getNamespaceResourceGroup(NamespaceName.get(namespaceName)))); admin.namespaces().removeNamespaceResourceGroup(namespaceName); Awaitility.await().untilAsserted(() -> - assertNull(pulsar.getResourceGroupServiceManager().getNamespaceResourceGroup(namespaceName))); + assertNull(pulsar.getResourceGroupServiceManager() + .getNamespaceResourceGroup(NamespaceName.get(namespaceName)))); admin.namespaces().deleteNamespace(namespaceName); deleteResourceGroup(rgName); @@ -178,15 +179,13 @@ public void testResourceGroupUpdateLoop() throws PulsarAdminException { pulsar.getPulsarResources().getResourcegroupResources().getStore().registerListener( notification -> { String notifyPath = notification.getPath(); - String rgName = notifyPath.substring(notifyPath.lastIndexOf('/') + 1); - if (!notifyPath.startsWith(path(RESOURCEGROUPS))) { - return; - } - if (RESOURCEGROUPS.equals(rgName)) { + if (!ResourceGroupResources.isResourceGroupPath(notifyPath)) { return; } + + String rgName = ResourceGroupResources.resourceGroupNameFromPath(notifyPath).get(); pulsar.getPulsarResources().getResourcegroupResources() - .getAsync(notifyPath).whenComplete((optionalRg, ex) -> { + .getResourceGroupAsync(rgName).whenComplete((optionalRg, ex) -> { if (ex != null) { return; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java index b51ebf6346c81..382c2a5533d23 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java @@ -21,11 +21,13 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import javax.naming.Name; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.naming.NamespaceName; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -86,7 +88,7 @@ public void testRateLimit(String topicString) throws PulsarAdminException, Pulsa Awaitility.await().untilAsserted(() -> assertNotNull(pulsar.getResourceGroupServiceManager() - .getNamespaceResourceGroup(namespaceName))); + .getNamespaceResourceGroup(NamespaceName.get(namespaceName)))); Awaitility.await().untilAsserted(() -> assertNotNull(pulsar.getResourceGroupServiceManager() 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 d3471dcd71f8b..c1d781f3920be 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 @@ -25,6 +25,7 @@ import org.apache.pulsar.broker.service.resource.usage.NetworkUsage; import org.apache.pulsar.broker.service.resource.usage.ResourceUsage; 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.slf4j.Logger; @@ -113,7 +114,7 @@ public void measureOpsTime() throws PulsarAdminException { final String tenantName = "SomeTenant"; final String namespaceName = "SomeNameSpace"; rgs.registerTenant(rgName, tenantName); - final String tenantAndNamespaceName = tenantName + "/" + namespaceName; + final NamespaceName tenantAndNamespaceName = NamespaceName.get(tenantName, namespaceName); rgs.registerNameSpace(rgName, tenantAndNamespaceName); mSecsStart = System.currentTimeMillis(); for (int ix = 0; ix < numPerfTestIterations; ix++) { @@ -195,9 +196,8 @@ public void testResourceGroupOps() throws PulsarAdminException, InterruptedExcep final String tenantName = topic.getTenant(); final String namespaceName = topic.getNamespacePortion(); rgs.registerTenant(rgName, tenantName); - // Registering with the (non-qualified) namespace should throw. - Assert.assertThrows(PulsarAdminException.class, () -> rgs.registerNameSpace(rgName, namespaceName)); - final String tenantAndNamespace = tenantName + "/" + namespaceName; + + final NamespaceName tenantAndNamespace = NamespaceName.get(tenantName, namespaceName); rgs.registerNameSpace(rgName, tenantAndNamespace); // Delete of our valid config should throw until we unref correspondingly. @@ -233,8 +233,6 @@ public void testResourceGroupOps() throws PulsarAdminException, InterruptedExcep } rgs.unRegisterTenant(rgName, tenantName); - // Unregistering with the (non-qualified) namespace should throw. - Assert.assertThrows(PulsarAdminException.class, () -> rgs.unRegisterNameSpace(rgName, namespaceName)); rgs.unRegisterNameSpace(rgName, tenantAndNamespace); BytesAndMessagesCount publishQuota = rgs.getPublishRateLimiters(rgName); 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 3e4da9f6562b8..a89d759e7ab29 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 @@ -33,6 +33,7 @@ import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +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; @@ -137,7 +138,7 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { final String tenantString = myTopic.getTenant(); final String nsString = myTopic.getNamespace(); rgs.registerTenant(activeRgName, tenantString); - rgs.registerNameSpace(activeRgName, nsString); + rgs.registerNameSpace(activeRgName, NamespaceName.get(nsString)); final int NumMessagesToSend = 10; int sentNumBytes = 0; @@ -180,9 +181,9 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { true, true); consumer.close(); - + rgs.unRegisterTenant(activeRgName, tenantString); - rgs.unRegisterNameSpace(activeRgName, nsString); + rgs.unRegisterNameSpace(activeRgName, NamespaceName.get(nsString)); rgs.resourceGroupDelete(activeRgName); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerConfigurationTest.java index d05b5a050844e..f4e6421e246e9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerConfigurationTest.java @@ -21,6 +21,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.cache.ConfigurationCacheService; +import org.apache.pulsar.broker.resources.PulsarResources; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -80,6 +81,7 @@ private PulsarService getPulsarService() { ConfigurationCacheService configurationCacheService = mock(ConfigurationCacheService.class); when(pulsarService.getConfiguration()).thenReturn(serviceConfiguration); when(pulsarService.getConfigurationCache()).thenReturn(configurationCacheService); + when(pulsarService.getPulsarResources()).thenReturn(mock(PulsarResources.class)); return pulsarService; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java index 16e313062cd6a..9ecb2fb459061 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.service.BrokerService.BROKER_SERVICE_CONFIGURATION_PATH; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; @@ -33,6 +32,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -311,14 +311,10 @@ private boolean areAllConsumersConnected(List> consumers) { } private void upsertLookupPermits(int permits) throws Exception { - Map throttlingMap = Maps.newHashMap(); - throttlingMap.put("maxConcurrentLookupRequest", Integer.toString(permits)); - byte[] content = ObjectMapperFactory.getThreadLocal().writeValueAsBytes(throttlingMap); - if (mockZooKeeper.exists(BROKER_SERVICE_CONFIGURATION_PATH, false) != null) { - mockZooKeeper.setData(BROKER_SERVICE_CONFIGURATION_PATH, content, -1); - } else { - ZkUtils.createFullPathOptimistic(mockZooKeeper, BROKER_SERVICE_CONFIGURATION_PATH, content, - ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } + pulsar.getPulsarResources().getDynamicConfigResources().setDynamicConfigurationWithCreate(optMap -> { + Map map = optMap.orElse(new TreeMap<>()); + map.put("maxConcurrentLookupRequest", Integer.toString(permits)); + return map; + }); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 43a22f470a2bd..229e5fc5f709b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -68,6 +68,7 @@ import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -152,6 +153,9 @@ public void setup() throws Exception { doReturn(store).when(pulsar).getLocalMetadataStore(); doReturn(store).when(pulsar).getConfigurationMetadataStore(); + PulsarResources pulsarResources = new PulsarResources(store, store); + doReturn(pulsarResources).when(pulsar).getPulsarResources(); + brokerService = spy(new BrokerService(pulsar, eventLoopGroup)); doReturn(brokerService).when(pulsar).getBrokerService(); 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 ad9f7c78ee5b8..a41f4d153cae1 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 @@ -91,10 +91,10 @@ import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.persistent.CompactorSubscription; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; @@ -117,6 +117,7 @@ import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.ProducerAccessMode; import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.protocol.ByteBufPair; @@ -129,6 +130,7 @@ import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.zookeeper.ZooKeeperCache; import org.apache.pulsar.zookeeper.ZooKeeperDataCache; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.zookeeper.ZooKeeper; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -215,6 +217,8 @@ public void setup() throws Exception { store = new ZKMetadataStore(mockZk); PulsarResources pulsarResources = spy(new PulsarResources(store, store)); + NamespaceResources nsr = spy(new NamespaceResources(store, store, 30)); + doReturn(nsr).when(pulsarResources).getNamespaceResources(); doReturn(pulsarResources).when(pulsar).getPulsarResources(); doReturn(store).when(pulsar).getLocalMetadataStore(); @@ -1873,9 +1877,10 @@ public void testCompactionTriggeredAfterThresholdFirstInvocation() throws Except Policies policies = new Policies(); policies.compaction_threshold = 1L; - when(pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(successTopicName).getNamespace()))) - .thenReturn(Optional.of(policies)); + + NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); + NamespaceName ns = TopicName.get(successTopicName).getNamespaceObject(); + doReturn(Optional.of(policies)).when(nsr).getPolicies(ns); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); @@ -1906,9 +1911,10 @@ public void testCompactionTriggeredAfterThresholdSecondInvocation() throws Excep Policies policies = new Policies(); policies.compaction_threshold = 1L; - when(pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(successTopicName).getNamespace()))) - .thenReturn(Optional.of(policies)); + + NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); + NamespaceName ns = TopicName.get(successTopicName).getNamespaceObject(); + doReturn(Optional.of(policies)).when(nsr).getPolicies(ns); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); @@ -1934,9 +1940,10 @@ public void testCompactionDisabledWithZeroThreshold() throws Exception { Policies policies = new Policies(); policies.compaction_threshold = 0L; - when(pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(successTopicName).getNamespace()))) - .thenReturn(Optional.of(policies)); + + NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); + NamespaceName ns = TopicName.get(successTopicName).getNamespaceObject(); + doReturn(Optional.of(policies)).when(nsr).getPolicies(ns); doReturn(1000L).when(ledgerMock).getEstimatedBacklogSize(); @@ -2085,9 +2092,9 @@ public void testCheckInactiveSubscriptions() throws Exception { 50000, serverCnx, "app1", Collections.emptyMap(), false, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, nonDeletableSubscription1, consumer); - when(pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(successTopicName).getNamespace()))) - .thenReturn(Optional.of(new Policies())); + NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); + NamespaceName ns = TopicName.get(successTopicName).getNamespaceObject(); + doReturn(Optional.of(new Policies())).when(nsr).getPolicies(ns); ServiceConfiguration svcConfig = spy(new ServiceConfiguration()); doReturn(5).when(svcConfig).getSubscriptionExpirationTimeMinutes(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRemoveClusterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRemoveClusterTest.java new file mode 100644 index 0000000000000..701ab47fa7f69 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRemoveClusterTest.java @@ -0,0 +1,110 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import com.google.common.collect.Sets; +import java.lang.reflect.Method; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +/** + * Starts 3 brokers that are in 3 different clusters + */ +@Test(groups = "broker") +public class ReplicatorRemoveClusterTest extends ReplicatorTestBase { + + protected String methodName; + + @BeforeMethod(alwaysRun = true) + public void beforeMethod(Method m) throws Exception { + methodName = m.getName(); + admin1.namespaces().removeBacklogQuota("pulsar/ns"); + admin1.namespaces().removeBacklogQuota("pulsar/ns1"); + admin1.namespaces().removeBacklogQuota("pulsar/global/ns"); + } + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @DataProvider(name = "partitionedTopic") + public Object[][] partitionedTopicProvider() { + return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; + } + + + @Test + public void testRemoveClusterFromNamespace() throws Exception { + admin1.tenants().createTenant("pulsar1", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), + Sets.newHashSet("r1", "r2", "r3"))); + + admin1.namespaces().createNamespace("pulsar1/ns1", Sets.newHashSet("r1", "r2", "r3")); + + PulsarClient repClient1 = pulsar1.getBrokerService().getReplicationClient("r3"); + Assert.assertNotNull(repClient1); + Assert.assertFalse(repClient1.isClosed()); + + PulsarClient client = PulsarClient.builder() + .serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + + final String topicName = "persistent://pulsar1/ns1/testRemoveClusterFromNamespace-" + UUID.randomUUID(); + + Producer producer = client.newProducer() + .topic(topicName) + .create(); + + producer.send("Pulsar".getBytes()); + + producer.close(); + client.close(); + + Replicator replicator = pulsar1.getBrokerService().getTopicReference(topicName) + .get().getReplicators().get("r3"); + + Awaitility.await().untilAsserted(() -> Assert.assertTrue(replicator.isConnected())); + + admin1.clusters().deleteCluster("r3"); + + Awaitility.await().untilAsserted(() -> Assert.assertFalse(replicator.isConnected())); + Awaitility.await().untilAsserted(() -> Assert.assertTrue(repClient1.isClosed())); + + Awaitility.await().untilAsserted(() -> Assert.assertNull( + pulsar1.getBrokerService().getReplicationClients().get("r3"))); + } +} 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 527fd2c28ff63..9b4fcc87aa186 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 @@ -1235,55 +1235,6 @@ public void createPartitionedTopicTest() throws Exception { checkListContainExpectedTopic(admin3, namespace, expectedTopicList); } - @Test - public void testRemoveClusterFromNamespace() throws Exception { - final String cluster4 = "r4"; - admin1.clusters().createCluster(cluster4, ClusterData.builder() - .serviceUrl(url3.toString()) - .serviceUrlTls(urlTls3.toString()) - .brokerServiceUrl(pulsar3.getSafeBrokerServiceUrl()) - .brokerServiceUrlTls(pulsar3.getBrokerServiceUrlTls()) - .build()); - - admin1.tenants().createTenant("pulsar1", - new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), - Sets.newHashSet("r1", "r3", cluster4))); - - admin1.namespaces().createNamespace("pulsar1/ns1", Sets.newHashSet("r1", "r3", cluster4)); - - PulsarClient repClient1 = pulsar1.getBrokerService().getReplicationClient(cluster4); - Assert.assertNotNull(repClient1); - Assert.assertFalse(repClient1.isClosed()); - - PulsarClient client = PulsarClient.builder() - .serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) - .build(); - - final String topicName = "persistent://pulsar1/ns1/testRemoveClusterFromNamespace-" + UUID.randomUUID(); - - Producer producer = client.newProducer() - .topic(topicName) - .create(); - - producer.send("Pulsar".getBytes()); - - producer.close(); - client.close(); - - Replicator replicator = pulsar1.getBrokerService().getTopicReference(topicName) - .get().getReplicators().get(cluster4); - - Awaitility.await().untilAsserted(() -> Assert.assertTrue(replicator.isConnected())); - - admin1.clusters().deleteCluster(cluster4); - - Awaitility.await().untilAsserted(() -> Assert.assertFalse(replicator.isConnected())); - Awaitility.await().untilAsserted(() -> Assert.assertTrue(repClient1.isClosed())); - - Awaitility.await().untilAsserted(() -> Assert.assertNull( - pulsar1.getBrokerService().getReplicationClients().get(cluster4))); - } - @Test public void testDoNotReplicateSystemTopic() throws Exception { final String namespace = newUniqueName("pulsar/ns"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 8108798859218..6376c70d4c4a8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -164,6 +164,7 @@ public void setup() throws Exception { svcConfig.setKeepAliveIntervalSeconds(inSec(1, TimeUnit.SECONDS)); svcConfig.setBacklogQuotaCheckEnabled(false); doReturn(svcConfig).when(pulsar).getConfiguration(); + doReturn(mock(PulsarResources.class)).when(pulsar).getPulsarResources(); doReturn("use").when(svcConfig).getClusterName(); @@ -1349,7 +1350,7 @@ public void testProducerFailureOnEncryptionRequiredTopic() throws Exception { // `org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.getPoliciesDispatchRate` policies.clusterDispatchRate = Maps.newHashMap(); doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(namespaceResources) - .getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); + .getPoliciesAsync(TopicName.get(encryptionRequiredTopicName).getNamespaceObject()); // test failure case: unencrypted producer cannot connect ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 2 /* producer id */, 2 /* request id */, @@ -1453,7 +1454,8 @@ public void testSendFailureOnEncryptionRequiredTopic() throws Exception { // add `clusterDispatchRate` otherwise there will be a NPE // `org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.getPoliciesDispatchRate` policies.clusterDispatchRate = Maps.newHashMap(); - doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(namespaceResources).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); + doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(namespaceResources) + .getPoliciesAsync(TopicName.get(encryptionRequiredTopicName).getNamespaceObject()); ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 1 /* producer id */, 1 /* request id */, "prod-name", true, Collections.emptyMap()); 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 5b1bb6282a5b7..b4c80ef273f11 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 @@ -56,6 +56,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Consumer; @@ -122,6 +123,7 @@ public void setup() throws Exception { svcConfig.setBrokerShutdownTimeoutMs(0L); svcConfig.setTransactionCoordinatorEnabled(true); pulsarMock = spy(new PulsarService(svcConfig)); + doReturn(mock(PulsarResources.class)).when(pulsarMock).getPulsarResources(); doReturn(new InMemTransactionBufferProvider()).when(pulsarMock).getTransactionBufferProvider(); doReturn(new TransactionPendingAckStoreProvider() { @Override 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 48bf6025675ec..30f75389744bb 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 @@ -285,8 +285,7 @@ public void testMaxRequestSize() throws Exception { // Create local cluster String localCluster = "test"; - String clusterPath = PulsarWebResource.path("clusters", localCluster); - pulsar.getPulsarResources().getClusterResources().create(clusterPath, ClusterDataImpl.builder().build()); + pulsar.getPulsarResources().getClusterResources().createCluster(localCluster, ClusterDataImpl.builder().build()); TenantInfo info2 = TenantInfo.builder() .adminRoles(Collections.singleton(StringUtils.repeat("*", 1 * 1024))) .allowedClusters(Sets.newHashSet(localCluster)) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java index b553d6bc07874..c4b569c36d90e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java @@ -18,6 +18,14 @@ */ package org.apache.pulsar.broker.zookeeper; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; import java.nio.charset.Charset; import java.util.Arrays; import java.util.HashSet; @@ -28,10 +36,8 @@ import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.pulsar.PulsarClusterMetadataSetup; import org.apache.pulsar.PulsarInitialNamespaceSetup; -import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.TenantResources; -import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; @@ -43,16 +49,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.net.InetSocketAddress; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; - @Slf4j @Test(groups = "broker") public class ClusterMetadataSetupTest { @@ -184,9 +180,8 @@ public void testInitialNamespaceSetup() throws Exception { MetadataStoreConfig.builder().build())) { TenantResources tenantResources = new TenantResources(store, PulsarResources.DEFAULT_OPERATION_TIMEOUT_SEC); - List namespaces = tenantResources.getChildren(PulsarWebResource - .path(ConfigurationCacheService.POLICIES, "test")); - assertEquals(new HashSet<>(namespaces), new HashSet<>(Arrays.asList("a", "b", "c"))); + List namespaces = tenantResources.getListOfNamespaces("test"); + assertEquals(new HashSet<>(namespaces), new HashSet<>(Arrays.asList("test/a", "test/b", "test/c"))); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionCreationTest.java index 782c9fcf0e328..da9ece0b4ecf0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionCreationTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.client.api; -import org.apache.pulsar.broker.admin.ZkAdminPaths; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.common.naming.TopicDomain; @@ -122,10 +121,10 @@ public Object[] restCreateMissedPartitions() { public void testCreateMissedPartitions(boolean useRestApi) throws PulsarAdminException, PulsarClientException, MetadataStoreException { conf.setAllowAutoTopicCreation(false); final String topic = "testCreateMissedPartitions-useRestApi-" + useRestApi; - String path = ZkAdminPaths.partitionedTopicPath(TopicName.get(topic)); int numPartitions = 3; // simulate partitioned topic without partitions - pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().create(path, + pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .createPartitionedTopicAsync(TopicName.get(topic), new PartitionedTopicMetadata(numPartitions)); Consumer consumer = null; try { 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 a111dd8c9b8e9..8e385a62b1b9b 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 @@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.UUID.randomUUID; -import static org.apache.pulsar.broker.service.BrokerService.BROKER_SERVICE_CONFIGURATION_PATH; import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; @@ -648,14 +647,11 @@ public void testInvalidDynamicConfiguration() throws Exception { } // (3) restart broker with invalid config value - - MetadataCache> dynamicConfigurationCache = pulsar.getBrokerService() - .getDynamicConfigurationCache(); - Map configurationMap = dynamicConfigurationCache.get(BROKER_SERVICE_CONFIGURATION_PATH).get().get(); - configurationMap.put("loadManagerClassName", "org.apache.pulsar.invalid.loadmanager"); - byte[] content = ObjectMapperFactory.getThreadLocal().writeValueAsBytes(configurationMap); - dynamicConfigurationCache.invalidate(BROKER_SERVICE_CONFIGURATION_PATH); - mockZooKeeper.setData(BROKER_SERVICE_CONFIGURATION_PATH, content, -1); + pulsar.getPulsarResources().getDynamicConfigResources() + .setDynamicConfiguration(m -> { + m.put("loadManagerClassName", "org.apache.pulsar.invalid.loadmanager"); + return m; + }); } static class TimestampEntryCount { 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 9c9a0951c50e1..48752f3ae695e 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 @@ -19,8 +19,8 @@ package org.apache.pulsar.discovery.service; import static org.apache.bookkeeper.util.MathUtils.signSafeMod; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; - +import com.google.common.base.Joiner; +import io.netty.util.concurrent.DefaultThreadFactory; import java.io.Closeable; import java.io.IOException; import java.util.List; @@ -28,26 +28,20 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.resources.MetadataStoreCacheLoader; import org.apache.pulsar.broker.resources.PulsarResources; -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; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Joiner; - -import io.netty.util.concurrent.DefaultThreadFactory; - /** * Maintains available active broker list and returns next active broker in round-robin for discovery service. * @@ -105,10 +99,9 @@ CompletableFuture getPartitionedTopicMetadata(Discover CompletableFuture metadataFuture = new CompletableFuture<>(); try { checkAuthorization(service, topicName, role, authenticationData); - final String path = path(PARTITIONED_TOPIC_PATH_ZNODE, - topicName.getNamespaceObject().toString(), topicName.getDomain().value(), topicName.getEncodedLocalName()); // gets the number of partitions from the zk cache - pulsarResources.getNamespaceResources().getPartitionedTopicResources().getAsync(path) + pulsarResources.getNamespaceResources().getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(topicName) .thenAccept(metadata -> { // if the partitioned topic is not found in zk, then the topic // is not partitioned @@ -142,7 +135,7 @@ protected static void checkAuthorization(DiscoveryService service, TopicName top TenantInfo tenantInfo; try { tenantInfo = service.getPulsarResources().getTenantResources() - .get(path(POLICIES, topicName.getTenant())) + .getTenant(topicName.getTenant()) .orElseThrow(() -> new IllegalAccessException("Property does not exist")); } catch (NotFoundException e) { LOG.warn("Failed to get property admin data for non existing property {}", topicName.getTenant()); 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 f152adc90c267..2ac3237a73314 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 @@ -18,7 +18,6 @@ */ package org.apache.pulsar.functions.worker; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.common.policies.data.PoliciesUtil.getBundles; import com.fasterxml.jackson.core.JsonProcessingException; @@ -55,6 +54,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.naming.NamespaceName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.Policies; @@ -303,7 +303,7 @@ public void initInBroker(ServiceConfiguration brokerConfig, String namespace = workerConfig.getPulsarFunctionsNamespace(); String[] a = workerConfig.getPulsarFunctionsNamespace().split("/"); - String property = a[0]; + String tenant = a[0]; String cluster = workerConfig.getPulsarFunctionsCluster(); int[] ar = null; @@ -315,10 +315,10 @@ public void initInBroker(ServiceConfiguration brokerConfig, // create tenant for function worker service try { - NamedEntity.checkName(property); - pulsarResources.getTenantResources().create(PolicyPath.path(POLICIES, property), + NamedEntity.checkName(tenant); + pulsarResources.getTenantResources().createTenant(tenant, new TenantInfoImpl(Sets.newHashSet(workerConfig.getSuperUserRoles()), Sets.newHashSet(cluster))); - LOG.info("Created property {} for function worker", property); + LOG.info("Created tenant {} for function worker", tenant); } catch (AlreadyExistsException e) { LOG.debug("Failed to create already existing property {} for function worker service", cluster, e); } catch (IllegalArgumentException e) { @@ -336,9 +336,7 @@ public void initInBroker(ServiceConfiguration brokerConfig, .serviceUrl(workerConfig.getPulsarWebServiceUrl()) .brokerServiceUrl(workerConfig.getPulsarServiceUrl()) .build(); - pulsarResources.getClusterResources().create( - PolicyPath.path("clusters", cluster), - clusterData); + pulsarResources.getClusterResources().createCluster(cluster, clusterData); LOG.info("Created cluster {} for function worker", cluster); } catch (AlreadyExistsException e) { LOG.debug("Failed to create already existing cluster {} for function worker service", cluster, e); @@ -358,8 +356,7 @@ public void initInBroker(ServiceConfiguration brokerConfig, int defaultNumberOfBundles = brokerConfig.getDefaultNumberOfNamespaceBundles(); policies.bundles = getBundles(defaultNumberOfBundles); - configurationCacheService.policiesCache().invalidate(PolicyPath.path(POLICIES, namespace)); - pulsarResources.getNamespaceResources().create(PolicyPath.path(POLICIES, namespace), policies); + pulsarResources.getNamespaceResources().createPolicies(NamespaceName.get(namespace), policies); LOG.info("Created namespace {} for function worker service", namespace); } catch (AlreadyExistsException e) { LOG.debug("Failed to create already existing namespace {} for function worker service", namespace); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreException.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreException.java index d9ffda03addb1..3ec46627ae148 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreException.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreException.java @@ -103,6 +103,10 @@ public BadVersionException(String msg) { * Failed to de-serialize the metadata. */ public static class ContentDeserializationException extends MetadataStoreException { + public ContentDeserializationException(String msg, Throwable t) { + super(msg, t); + } + public ContentDeserializationException(Throwable t) { super(t); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index a1c29cccb30b4..092bce0a9b6ae 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -102,7 +102,8 @@ private CompletableFuture>> readValueFromStore(String return FutureUtils .value(Optional.of(new CacheGetResult<>(obj, optRes.get().getStat()))); } catch (Throwable t) { - return FutureUtils.exception(new ContentDeserializationException(t)); + return FutureUtils.exception(new ContentDeserializationException( + "Failed to deserialize payload for key '" + path + "'", t)); } }); } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java index ae8e13405bf01..5ec977a1dfc6e 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java @@ -19,7 +19,6 @@ package org.apache.pulsar.proxy.server; import static org.apache.bookkeeper.common.util.MathUtils.signSafeMod; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import java.io.Closeable; import java.io.IOException; @@ -36,6 +35,7 @@ import org.apache.pulsar.broker.resources.PulsarResources; 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.policies.data.loadbalancer.LoadManagerReport; import org.apache.zookeeper.KeeperException; @@ -100,9 +100,8 @@ CompletableFuture getPartitionedTopicMetadata(ProxySer CompletableFuture metadataFuture = new CompletableFuture<>(); try { checkAuthorization(service, topicName, role, authenticationData); - final String path = path(PARTITIONED_TOPIC_PATH_ZNODE, - topicName.getNamespaceObject().toString(), "persistent", topicName.getEncodedLocalName()); - pulsarResources.getNamespaceResources().getPartitionedTopicResources().getAsync(path) + pulsarResources.getNamespaceResources().getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(topicName) .thenAccept(metadata -> { // if the partitioned topic is not found in zk, then the topic // is not partitioned @@ -121,7 +120,7 @@ CompletableFuture getPartitionedTopicMetadata(ProxySer return metadataFuture; } - protected static void checkAuthorization(ProxyService service, TopicName topicName, String role, + protected void checkAuthorization(ProxyService service, TopicName topicName, String role, AuthenticationDataSource authenticationData) throws Exception { if (!service.getConfiguration().isAuthorizationEnabled() || service.getConfiguration().getSuperUserRoles().contains(role)) { @@ -132,14 +131,10 @@ protected static void checkAuthorization(ProxyService service, TopicName topicNa 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.getConfigurationCacheService().propertiesCache() - .get(path(POLICIES, topicName.getTenant())) + tenantInfo = pulsarResources.getTenantResources().getTenant(topicName.getTenant()) .orElseThrow(() -> new IllegalAccessException("Property does not exist")); - } catch (KeeperException.NoNodeException e) { - LOG.warn("Failed to get property admin data for non existing property {}", topicName.getTenant()); - throw new IllegalAccessException("Property does not exist"); } catch (Exception e) { LOG.error("Failed to get property admin data for property"); throw new IllegalAccessException(String.format("Failed to get property %s admin data due to %s", @@ -154,13 +149,6 @@ protected static void checkAuthorization(ProxyService service, TopicName topicNa } } - public static String path(String... parts) { - StringBuilder sb = new StringBuilder(); - sb.append("/admin/"); - Joiner.on('/').appendTo(sb, parts); - return sb.toString(); - } - @Override public void close() throws IOException { metadataStoreCacheLoader.close(); 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 1318e5c327530..119594a0adee7 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 @@ -226,9 +226,8 @@ private ClusterData retrieveClusterData() throws PulsarServerException { "Failed to retrieve Cluster data due to empty ConfigurationStoreServers"); } try { - String path = "/admin/clusters/" + config.getClusterName(); - return localCluster = pulsarResources.getClusterResources().get(path) - .orElseThrow(() -> new NotFoundException(path)); + return localCluster = pulsarResources.getClusterResources().getCluster(config.getClusterName()) + .orElseThrow(() -> new NotFoundException("Cluster " + config.getClusterName())); } catch (Exception e) { throw new PulsarServerException(e); }