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 51e6acec334b3..9a2b44a74208e 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 @@ -275,7 +275,8 @@ public void start() throws PulsarServerException { this.webService = new WebService(this); this.webService.addRestResources("/", "org.apache.pulsar.broker.web", false); - this.webService.addRestResources("/admin", "org.apache.pulsar.broker.admin", true); + this.webService.addRestResources("/admin", "org.apache.pulsar.broker.admin.v1", true); + this.webService.addRestResources("/admin/v2", "org.apache.pulsar.broker.admin.v2", true); this.webService.addRestResources("/lookup", "org.apache.pulsar.broker.lookup", true); this.webService.addServlet("/metrics", @@ -462,8 +463,7 @@ public void loadNamespaceDestinations(NamespaceBundle bundle) { List> persistentTopics = Lists.newArrayList(); long topicLoadStart = System.nanoTime(); - for (String topic : getNamespaceService().getListOfDestinations(nsName.getProperty(), nsName.getCluster(), - nsName.getLocalName())) { + for (String topic : getNamespaceService().getListOfDestinations(nsName)) { try { DestinationName dn = DestinationName.get(topic); if (bundle.includes(dn)) { 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 f04d15bb63d2e..b91c63ac6c298 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 @@ -52,6 +52,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.PropertyAdmin; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; +import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.zookeeper.ZooKeeperCache; import org.apache.pulsar.zookeeper.ZooKeeperCache.Deserializer; @@ -189,11 +190,22 @@ public void validatePoliciesReadOnlyAccess() { protected List getListOfNamespaces(String property) throws Exception { List namespaces = Lists.newArrayList(); - for (String cluster : globalZk().getChildren(path(POLICIES, property), false)) { + // this will return a cluster in v1 and a namespace in v2 + for (String clusterOrNamespace : globalZk().getChildren(path(POLICIES, property), false)) { // Then get the list of namespaces try { - for (String namespace : globalZk().getChildren(path(POLICIES, property, cluster), false)) { - namespaces.add(String.format("%s/%s/%s", property, cluster, namespace)); + final List children = globalZk().getChildren(path(POLICIES, property, clusterOrNamespace), false); + if (children == null || children.isEmpty()) { + String namespace = NamespaceName.get(property, 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 + if (globalZk().getData(path(POLICIES, namespace), false, null).length != 0) { + namespaces.add(namespace); + } + } else { + children.forEach(ns -> { + namespaces.add(NamespaceName.get(property, clusterOrNamespace, ns).toString()); + }); } } catch (KeeperException.NoNodeException e) { // A cluster was deleted between the 2 getChildren() calls, ignoring @@ -204,7 +216,56 @@ protected List getListOfNamespaces(String property) throws Exception { return namespaces; } - + protected NamespaceName namespaceName; + + protected void validateNamespaceName(String property, String namespace) { + try { + this.namespaceName = NamespaceName.get(property, namespace); + } catch (IllegalArgumentException e) { + log.warn("[{}] Failed to create namespace with invalid name {}", clientAppId(), namespace, e); + throw new RestException(Status.PRECONDITION_FAILED, "Namespace name is not valid"); + } + } + + @Deprecated + protected void validateNamespaceName(String property, String cluster, String namespace) { + try { + this.namespaceName = NamespaceName.get(property, cluster, namespace); + } catch (IllegalArgumentException e) { + log.warn("[{}] Failed to create namespace with invalid name {}", clientAppId(), namespace, e); + throw new RestException(Status.PRECONDITION_FAILED, "Namespace name is not valid"); + } + } + + protected DestinationName destinationName; + + protected void validateDestinationName(String property, String namespace, String encodedTopic) { + String topic = Codec.decode(encodedTopic); + try { + this.namespaceName = NamespaceName.get(property, namespace); + this.destinationName = DestinationName.get(domain(), namespaceName, topic); + } catch (IllegalArgumentException e) { + log.warn("[{}] Failed to validate topic name {}://{}/{}/{}", clientAppId(), domain(), property, namespace, + topic, e); + throw new RestException(Status.PRECONDITION_FAILED, "Topic name is not valid"); + } + + this.destinationName = DestinationName.get(domain(), namespaceName, topic); + } + + @Deprecated + protected void validateDestinationName(String property, String cluster, String namespace, String encodedTopic) { + String topic = Codec.decode(encodedTopic); + try { + this.namespaceName = NamespaceName.get(property, cluster, namespace); + this.destinationName = DestinationName.get(domain(), namespaceName, topic); + } catch (IllegalArgumentException e) { + log.warn("[{}] Failed to validate topic name {}://{}/{}/{}/{}", clientAppId(), domain(), property, cluster, + namespace, topic, e); + throw new RestException(Status.PRECONDITION_FAILED, "Topic name is not valid"); + } + } + /** * Redirect the call to the specified broker * @@ -227,20 +288,20 @@ protected void validateBrokerName(String broker) throws MalformedURLException { } } - protected Policies getNamespacePolicies(String property, String cluster, String namespace) { + protected Policies getNamespacePolicies(NamespaceName namespaceName) { try { - Policies policies = policiesCache().get(AdminResource.path(POLICIES, property, cluster, namespace)) + Policies policies = policiesCache().get(AdminResource.path(POLICIES, namespaceName.toString())) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); // fetch bundles from LocalZK-policies NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory() - .getBundles(NamespaceName.get(property, cluster, namespace)); + .getBundles(namespaceName); BundlesData bundleData = NamespaceBundleFactory.getBundlesData(bundles); 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(), namespaceName, e); throw new RestException(e); } } @@ -249,27 +310,27 @@ public static ObjectMapper jsonMapper() { return ObjectMapperFactory.getThreadLocal(); } - ZooKeeperDataCache propertiesCache() { + public ZooKeeperDataCache propertiesCache() { return pulsar().getConfigurationCache().propertiesCache(); } - ZooKeeperDataCache policiesCache() { + protected ZooKeeperDataCache policiesCache() { return pulsar().getConfigurationCache().policiesCache(); } - ZooKeeperDataCache localPoliciesCache() { + protected ZooKeeperDataCache localPoliciesCache() { return pulsar().getLocalZkCacheService().policiesCache(); } - ZooKeeperDataCache clustersCache() { + protected ZooKeeperDataCache clustersCache() { return pulsar().getConfigurationCache().clustersCache(); } - ZooKeeperChildrenCache managedLedgerListCache() { + protected ZooKeeperChildrenCache managedLedgerListCache() { return pulsar().getLocalZkCacheService().managedLedgerListCache(); } - Set clusters() { + protected Set clusters() { try { return pulsar().getConfigurationCache().clustersListCache().get(); } catch (Exception e) { @@ -277,7 +338,7 @@ Set clusters() { } } - ZooKeeperChildrenCache clustersListCache() { + protected ZooKeeperChildrenCache clustersListCache() { return pulsar().getConfigurationCache().clustersListCache(); } @@ -297,32 +358,30 @@ protected ZooKeeperChildrenCache failureDomainListCache() { return pulsar().getConfigurationCache().failureDomainListCache(); } - protected PartitionedTopicMetadata getPartitionedTopicMetadata(String property, String cluster, String namespace, - String destination, boolean authoritative) { - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateClusterOwnership(dn.getCluster()); + protected PartitionedTopicMetadata getPartitionedTopicMetadata(DestinationName destinationName, + boolean authoritative) { + validateClusterOwnership(destinationName.getCluster()); // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can // serve/redirect request else fail partitioned-metadata-request so, client fails while creating // producer/consumer - validateGlobalNamespaceOwnership(dn.getNamespaceObject()); - + validateGlobalNamespaceOwnership(destinationName.getNamespaceObject()); + try { - checkConnect(dn); + checkConnect(destinationName); } catch (WebApplicationException e) { - validateAdminAccessOnProperty(dn.getProperty()); + validateAdminAccessOnProperty(destinationName.getProperty()); } catch (Exception e) { // unknown error marked as internal server error - log.warn("Unexpected error while authorizing lookup. destination={}, role={}. Error: {}", destination, + log.warn("Unexpected error while authorizing lookup. destination={}, role={}. Error: {}", destinationName, clientAppId(), e.getMessage(), e); throw new RestException(e); } - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(), - dn.getEncodedLocalName()); + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), destinationName.getEncodedLocalName()); PartitionedTopicMetadata partitionMetadata = fetchPartitionedTopicMetadata(pulsar(), path); if (log.isDebugEnabled()) { - log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), dn, + log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), destinationName, partitionMetadata.partitions); } return partitionMetadata; @@ -339,8 +398,8 @@ protected static PartitionedTopicMetadata fetchPartitionedTopicMetadata(PulsarSe } } - protected static CompletableFuture fetchPartitionedTopicMetadataAsync(PulsarService pulsar, - String path) { + protected static CompletableFuture fetchPartitionedTopicMetadataAsync( + PulsarService pulsar, String path) { CompletableFuture metadataFuture = new CompletableFuture<>(); try { // gets the number of partitions from the zk cache @@ -375,4 +434,22 @@ protected void validateClusterExists(String cluster) { throw new RestException(e); } } + + protected Policies getNamespacePolicies(String property, String cluster, String namespace) { + try { + Policies policies = policiesCache().get(AdminResource.path(POLICIES, property, cluster, namespace)) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); + // fetch bundles from LocalZK-policies + NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory() + .getBundles(NamespaceName.get(property, cluster, namespace)); + BundlesData bundleData = NamespaceBundleFactory.getBundlesData(bundles); + 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); + throw new RestException(e); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java deleted file mode 100644 index e2467f90bf2cf..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java +++ /dev/null @@ -1,1757 +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 static com.google.common.base.Preconditions.checkArgument; -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.broker.cache.ConfigurationCacheService.POLICIES_ROOT; -import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; - -import java.net.URI; -import java.net.URL; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Optional; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - -import javax.ws.rs.Consumes; -import javax.ws.rs.DELETE; -import javax.ws.rs.DefaultValue; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.PUT; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; -import javax.ws.rs.core.UriBuilder; - -import org.apache.pulsar.broker.PulsarServerException; -import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; -import org.apache.pulsar.broker.service.Subscription; -import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.service.persistent.PersistentReplicator; -import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.web.RestException; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.naming.DestinationName; -import org.apache.pulsar.common.naming.NamedEntity; -import org.apache.pulsar.common.naming.NamespaceBundle; -import org.apache.pulsar.common.naming.NamespaceBundleFactory; -import org.apache.pulsar.common.naming.NamespaceBundles; -import org.apache.pulsar.common.naming.NamespaceName; -import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.BacklogQuota; -import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; -import org.apache.pulsar.common.policies.data.BundlesData; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.DispatchRate; -import org.apache.pulsar.common.policies.data.PersistencePolicies; -import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; -import org.apache.pulsar.common.util.FutureUtil; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.collect.Sets.SetView; - -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiOperation; -import io.swagger.annotations.ApiResponse; -import io.swagger.annotations.ApiResponses; - -@Path("/namespaces") -@Produces(MediaType.APPLICATION_JSON) -@Consumes(MediaType.APPLICATION_JSON) -@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces") -public class Namespaces extends AdminResource { - - public static final String GLOBAL_CLUSTER = "global"; - private static final long MAX_BUNDLES = ((long) 1) << 32; - - @GET - @Path("/{property}") - @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property doesn't exist") }) - public List getPropertyNamespaces(@PathParam("property") String property) { - validateAdminAccessOnProperty(property); - - try { - return getListOfNamespaces(property); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get namespace list for propery: {} - Does not exist", clientAppId(), property); - throw new RestException(Status.NOT_FOUND, "Property does not exist"); - } catch (Exception e) { - log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}") - @ApiOperation(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, - @PathParam("cluster") String cluster) { - validateAdminAccessOnProperty(property); - List namespaces = Lists.newArrayList(); - if (!clusters().contains(cluster)) { - log.warn("[{}] Failed to get namespace list for property: {}/{} - Cluster does not exist", clientAppId(), - property, cluster); - throw new RestException(Status.NOT_FOUND, "Cluster does not exist"); - } - - try { - for (String namespace : globalZk().getChildren(path(POLICIES, property, cluster), false)) { - namespaces.add(String.format("%s/%s/%s", property, cluster, namespace)); - } - } catch (KeeperException.NoNodeException e) { - // NoNode means there are no namespaces for this property on the specified cluster, returning empty list - } catch (Exception e) { - log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e); - throw new RestException(e); - } - - namespaces.sort(null); - return namespaces; - } - - @GET - @Path("/{property}/{cluster}/{namespace}/destinations") - @ApiOperation(value = "Get the list of all the destinations under a certain namespace.", response = String.class, responseContainer = "Set") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public List getDestinations(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - // Validate that namespace exists, throws 404 if it doesn't exist - getNamespacePolicies(property, cluster, namespace); - - try { - return pulsar().getNamespaceService().getListOfDestinations(property, cluster, namespace); - } catch (Exception e) { - log.error("Failed to get topics list for namespace {}/{}/{}", property, cluster, namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}") - @ApiOperation(value = "Get the dump all the policies specified for a namespace.", response = Policies.class) - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public Policies getPolicies(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - return getNamespacePolicies(property, cluster, namespace); - } - - @PUT - @Path("/{property}/{cluster}/{namespace}") - @ApiOperation(value = "Creates a new empty namespace with no policies attached.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Namespace already exists"), - @ApiResponse(code = 412, message = "Namespace name is not valid") }) - public void createNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, BundlesData initialBundles) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - // If the namespace is non global, make sure property has the access on the cluster. For global namespace, same - // check is made at the time of setting replication. - if (!cluster.equals(GLOBAL_CLUSTER)) { - validateClusterForProperty(property, cluster); - } - if (!clusters().contains(cluster)) { - log.warn("[{}] Failed to create namespace. Cluster {} does not exist", clientAppId(), cluster); - throw new RestException(Status.NOT_FOUND, "Cluster does not exist"); - } - try { - checkNotNull(propertiesCache().get(path(POLICIES, property))); - } catch (NoNodeException nne) { - log.warn("[{}] Failed to create namespace. Property {} does not exist", clientAppId(), property); - throw new RestException(Status.NOT_FOUND, "Property does not exist"); - } catch (RestException e) { - throw e; - } catch (Exception e) { - throw new RestException(e); - } - try { - NamedEntity.checkName(namespace); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - Policies policies = new Policies(); - if (initialBundles != null && initialBundles.getNumBundles() > 0) { - if (initialBundles.getBoundaries() == null || initialBundles.getBoundaries().size() == 0) { - policies.bundles = getBundles(initialBundles.getNumBundles()); - } else { - policies.bundles = validateBundlesData(initialBundles); - } - } else { - int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles(); - policies.bundles = getBundles(defaultNumberOfBundles); - } - - zkCreateOptimistic(path(POLICIES, property, cluster, namespace), - jsonMapper().writeValueAsBytes(policies)); - log.info("[{}] Created namespace {}/{}/{}", clientAppId(), property, cluster, namespace); - } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create namespace {}/{}/{} - already exists", clientAppId(), property, cluster, - namespace); - throw new RestException(Status.CONFLICT, "Namespace already exists"); - } catch (IllegalArgumentException e) { - log.warn("[{}] Failed to create namespace with invalid name {}", clientAppId(), property, e); - throw new RestException(Status.PRECONDITION_FAILED, "Namespace name is not valid"); - } catch (Exception e) { - log.error("[{}] Failed to create namespace {}/{}/{}", clientAppId(), property, cluster, namespace, e); - throw new RestException(e); - } - } - - @DELETE - @Path("/{property}/{cluster}/{namespace}") - @ApiOperation(value = "Delete a namespace and all the destinations under it.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Namespace is not empty") }) - public void deleteNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - // ensure that non-global namespace is directed to the correct cluster - validateClusterOwnership(cluster); - - Entry policiesNode = null; - Policies policies = null; - - // ensure the local cluster is the only cluster for the global namespace configuration - try { - policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist.")); - - policies = policiesNode.getKey(); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - if (policies.replication_clusters.size() > 1) { - // There are still more than one clusters configured for the global namespace - throw new RestException(Status.PRECONDITION_FAILED, "Cannot delete the global namespace " + nsName - + ". There are still more than one replication clusters configured."); - } - if (policies.replication_clusters.size() == 1 - && !policies.replication_clusters.contains(config().getClusterName())) { - // the only replication cluster is other cluster, redirect - String replCluster = policies.replication_clusters.get(0); - ClusterData replClusterData = clustersCache().get(AdminResource.path("clusters", replCluster)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, - "Cluser " + replCluster + " does not exist")); - URL replClusterUrl; - if (!config().isTlsEnabled()) { - replClusterUrl = new URL(replClusterData.getServiceUrl()); - } else if (!replClusterData.getServiceUrlTls().isEmpty()) { - replClusterUrl = new URL(replClusterData.getServiceUrlTls()); - } else { - throw new RestException(Status.PRECONDITION_FAILED, - "The replication cluster does not provide TLS encrypted service"); - } - URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(replClusterUrl.getHost()) - .port(replClusterUrl.getPort()).replaceQueryParam("authoritative", false).build(); - log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, cluster); - throw new WebApplicationException(Response.temporaryRedirect(redirect).build()); - } - } - } catch (WebApplicationException wae) { - throw wae; - } catch (Exception e) { - throw new RestException(e); - } - - List destinations = getDestinations(property, cluster, namespace); - if (!destinations.isEmpty()) { - log.info("Found destinations: {}", destinations); - throw new RestException(Status.CONFLICT, "Cannot delete non empty namespace"); - } - - // set the policies to deleted so that somebody else cannot acquire this namespace - try { - policies.deleted = true; - globalZk().setData(path(POLICIES, property, cluster, namespace), jsonMapper().writeValueAsBytes(policies), - policiesNode.getValue().getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - } catch (Exception e) { - log.error("[{}] Failed to delete namespace on global ZK {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } - - // remove from owned namespace map and ephemeral node from ZK - try { - NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName); - for (NamespaceBundle bundle : bundles.getBundles()) { - // check if the bundle is owned by any broker, if not then we do not need to delete the bundle - if (pulsar().getNamespaceService().getOwner(bundle).isPresent()) { - pulsar().getAdminClient().namespaces().deleteNamespaceBundle(nsName.toString(), - bundle.getBundleRange()); - } - } - - // we have successfully removed all the ownership for the namespace, the policies znode can be deleted now - final String globalZkPolicyPath = path(POLICIES, property, cluster, namespace); - final String lcaolZkPolicyPath = joinPath(LOCAL_POLICIES_ROOT, property, cluster, namespace); - globalZk().delete(globalZkPolicyPath, -1); - localZk().delete(lcaolZkPolicyPath, -1); - policiesCache().invalidate(globalZkPolicyPath); - localCacheService().policiesCache().invalidate(lcaolZkPolicyPath); - } catch (PulsarAdminException cae) { - throw new RestException(cae); - } catch (Exception e) { - log.error(String.format("[%s] Failed to remove owned namespace %s/%s/%s", clientAppId(), property, cluster, - namespace), e); - // avoid throwing exception in case of the second failure - } - - } - - @DELETE - @Path("/{property}/{cluster}/{namespace}/{bundle}") - @ApiOperation(value = "Delete a namespace bundle and all the destinations under it.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Namespace bundle is not empty") }) - public void deleteNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - // ensure that non-global namespace is directed to the correct cluster - validateClusterOwnership(cluster); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - // ensure the local cluster is the only cluster for the global namespace configuration - try { - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - if (policies.replication_clusters.size() > 1) { - // There are still more than one clusters configured for the global namespace - throw new RestException(Status.PRECONDITION_FAILED, "Cannot delete the global namespace " + nsName - + ". There are still more than one replication clusters configured."); - } - if (policies.replication_clusters.size() == 1 - && !policies.replication_clusters.contains(config().getClusterName())) { - // the only replication cluster is other cluster, redirect - String replCluster = policies.replication_clusters.get(0); - ClusterData replClusterData = clustersCache().get(AdminResource.path("clusters", replCluster)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, - "Cluser " + replCluster + " does not exist")); - URL replClusterUrl; - if (!config().isTlsEnabled()) { - replClusterUrl = new URL(replClusterData.getServiceUrl()); - } else if (!replClusterData.getServiceUrlTls().isEmpty()) { - replClusterUrl = new URL(replClusterData.getServiceUrlTls()); - } else { - throw new RestException(Status.PRECONDITION_FAILED, - "The replication cluster does not provide TLS encrypted service"); - } - URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(replClusterUrl.getHost()) - .port(replClusterUrl.getPort()).replaceQueryParam("authoritative", false).build(); - log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, cluster); - throw new WebApplicationException(Response.temporaryRedirect(redirect).build()); - } - } - } catch (WebApplicationException wae) { - throw wae; - } catch (Exception e) { - throw new RestException(e); - } - - NamespaceBundle bundle = validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, - true); - try { - List destinations = getDestinations(property, cluster, namespace); - for (String destination : destinations) { - NamespaceBundle destinationBundle = (NamespaceBundle) pulsar().getNamespaceService() - .getBundle(DestinationName.get(destination)); - if (bundle.equals(destinationBundle)) { - throw new RestException(Status.CONFLICT, "Cannot delete non empty bundle"); - } - } - - // remove from owned namespace map and ephemeral node from ZK - pulsar().getNamespaceService().removeOwnedServiceUnit(bundle); - } catch (WebApplicationException wae) { - throw wae; - } catch (Exception e) { - log.error("[{}] Failed to remove namespace bundle {}/{}", clientAppId(), nsName.toString(), bundleRange, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/permissions") - @ApiOperation(value = "Retrieve the permissions for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Namespace is not empty") }) - public Map> getPermissions(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - return policies.auth_policies.namespace_auth; - } - - @POST - @Path("/{property}/{cluster}/{namespace}/permissions/{role}") - @ApiOperation(value = "Grant a new permission to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void grantPermissionOnNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("role") String role, Set actions) { - validateAdminAccessOnProperty(property); - - NamespaceName namespaceName = NamespaceName.get(property, cluster, namespace); - try { - pulsar().getBrokerService().getAuthorizationService() - .grantPermissionAsync(namespaceName, actions, role, null/*additional auth-data json*/) - .get(); - } catch (InterruptedException e) { - log.error("[{}] Failed to get permissions for namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } catch (ExecutionException e) { - if (e.getCause() instanceof IllegalArgumentException) { - log.warn("[{}] Failed to set permissions for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } else if (e.getCause() instanceof IllegalStateException) { - log.warn("[{}] Failed to set permissions for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } else { - log.error("[{}] Failed to get permissions for namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } - } - } - - @DELETE - @Path("/{property}/{cluster}/{namespace}/permissions/{role}") - @ApiOperation(value = "Revoke all permissions to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public void revokePermissionsOnNamespace(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("role") String role) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - try { - Stat nodeStat = new Stat(); - byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - policies.auth_policies.namespace_auth.remove(role); - - // Write back the new policies into zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), jsonMapper().writeValueAsBytes(policies), - nodeStat.getVersion()); - - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully revoked access for role {} - namespace {}/{}/{}", clientAppId(), role, property, - cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to revoke permissions for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to revoke permissions on namespace {}/{}/{}: concurrent modification", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to revoke permissions on namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/replication") - @ApiOperation(value = "Get the replication clusters for a namespace.", response = String.class, responseContainer = "List") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 412, message = "Namespace is not global") }) - public List getNamespaceReplicationClusters(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - if (!cluster.equals("global")) { - throw new RestException(Status.PRECONDITION_FAILED, - "Cannot get the replication clusters for a non-global namespace"); - } - - Policies policies = getNamespacePolicies(property, cluster, namespace); - return policies.replication_clusters; - } - - @POST - @Path("/{property}/{cluster}/{namespace}/replication") - @ApiOperation(value = "Set the replication clusters for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"), - @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") }) - public void setNamespaceReplicationClusters(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, List clusterIds) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - Set replicationClusterSet = Sets.newHashSet(clusterIds); - if (!cluster.equals("global")) { - throw new RestException(Status.PRECONDITION_FAILED, "Cannot set replication on a non-global namespace"); - } - - if (replicationClusterSet.contains("global")) { - throw new RestException(Status.PRECONDITION_FAILED, - "Cannot specify global in the list of replication clusters"); - } - - Set clusters = clusters(); - for (String clusterId : replicationClusterSet) { - if (!clusters.contains(clusterId)) { - throw new RestException(Status.FORBIDDEN, "Invalid cluster id: " + clusterId); - } - validatePeerClusterConflict(clusterId, replicationClusterSet); - } - - for (String clusterId : replicationClusterSet) { - validateClusterForProperty(property, clusterId); - } - - Entry policiesNode = null; - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - - try { - // Force to read the data s.t. the watch to the cache content is setup. - policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); - policiesNode.getKey().replication_clusters = clusterIds; - - // Write back the new policies into zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), - jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - - log.info("[{}] Successfully updated the replication clusters on namespace {}/{}/{}", clientAppId(), - property, cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update the replication clusters for namespace {}/{}/{}: does not exist", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn( - "[{}] Failed to update the replication clusters on namespace {}/{}/{} expected policy node version={} : concurrent modification", - clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); - - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to update the replication clusters on namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/messageTTL") - @ApiOperation(value = "Get the message TTL for the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public int getNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - return policies.message_ttl_in_seconds; - } - - @POST - @Path("/{property}/{cluster}/{namespace}/messageTTL") - @ApiOperation(value = "Set message TTL in seconds for namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 412, message = "Invalid TTL") }) - public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, int messageTTL) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - if (messageTTL < 0) { - throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL"); - } - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - Entry policiesNode = null; - - try { - // Force to read the data s.t. the watch to the cache content is setup. - policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); - policiesNode.getKey().message_ttl_in_seconds = messageTTL; - - // Write back the new policies into zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), - jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - - log.info("[{}] Successfully updated the message TTL on namespace {}/{}/{}", clientAppId(), property, - cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update the message TTL for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn( - "[{}] Failed to update the message TTL on namespace {}/{}/{} expected policy node version={} : concurrent modification", - clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); - - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to update the message TTL on namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } - } - - @POST - @Path("/{property}/{cluster}/{namespace}/antiAffinity") - @ApiOperation(value = "Set anti-affinity group for a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 412, message = "Invalid antiAffinityGroup") }) - public void setNamespaceAntiAffinityGroup(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, String antiAffinityGroup) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - log.info("[{}] Setting anti-affinity group {} for {}/{}/{}", clientAppId(), antiAffinityGroup, property, - cluster, namespace); - - if (isBlank(antiAffinityGroup)) { - throw new RestException(Status.PRECONDITION_FAILED, "antiAffinityGroup can't be empty"); - } - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - Entry policiesNode = null; - - try { - // Force to read the data s.t. the watch to the cache content is setup. - policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); - policiesNode.getKey().antiAffinityGroup = antiAffinityGroup; - - // Write back the new policies into zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), - jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - - log.info("[{}] Successfully updated the antiAffinityGroup {} on namespace {}/{}/{}", clientAppId(), - antiAffinityGroup, property, cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update the antiAffinityGroup for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn( - "[{}] Failed to update the antiAffinityGroup on namespace {}/{}/{} expected policy node version={} : concurrent modification", - clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); - - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to update the antiAffinityGroup on namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/antiAffinity") - @ApiOperation(value = "Get anti-affinity group of a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public String getNamespaceAntiAffinityGroup(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - return getNamespacePolicies(property, cluster, namespace).antiAffinityGroup; - } - - @GET - @Path("{cluster}/antiAffinity/{group}") - @ApiOperation(value = "Get all namespaces that are grouped by given anti-affinity group in a given cluster. api can be only accessed by admin of any of the existing property") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 412, message = "Cluster not exist/Anti-affinity group can't be empty.") }) - public List getAntiAffinityNamespaces(@PathParam("cluster") String cluster, - @PathParam("group") String antiAffinityGroup, @QueryParam("property") String property) { - validateAdminAccessOnProperty(property); - - log.info("[{}]-{} Finding namespaces for {} in {}", clientAppId(), property, antiAffinityGroup, cluster); - - if (isBlank(antiAffinityGroup)) { - throw new RestException(Status.PRECONDITION_FAILED, "anti-affinity group can't be empty."); - } - validateClusterExists(cluster); - List namespaces = Lists.newArrayList(); - try { - for (String prop : globalZk().getChildren(POLICIES_ROOT, false)) { - for (String namespace : globalZk().getChildren(path(POLICIES, prop, cluster), false)) { - Optional policies = policiesCache() - .get(AdminResource.path(POLICIES, prop, cluster, namespace)); - if (policies.isPresent() && antiAffinityGroup.equalsIgnoreCase(policies.get().antiAffinityGroup)) { - namespaces.add(String.format("%s/%s/%s", prop, cluster, namespace)); - } - } - } - } catch (Exception e) { - log.warn("Failed to list of properties/namespace from global-zk", e); - } - return namespaces; - } - - @DELETE - @Path("/{property}/{cluster}/{namespace}/antiAffinity") - @ApiOperation(value = "Remove anti-affinity group of a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void removeNamespaceAntiAffinityGroup(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - log.info("[{}] Deleting anti-affinity group for {}/{}/{}", clientAppId(), property, cluster, namespace); - - try { - Stat nodeStat = new Stat(); - final String path = path(POLICIES, property, cluster, namespace); - byte[] content = globalZk().getData(path, null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - policies.antiAffinityGroup = null; - globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully removed anti-affinity group for a namespace={}/{}/{}", clientAppId(), property, - cluster, namespace); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - @POST - @Path("/{property}/{cluster}/{namespace}/deduplication") - @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public void modifyDeduplication(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, boolean enableDeduplication) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - Entry policiesNode = null; - - try { - // Force to read the data s.t. the watch to the cache content is setup. - policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); - policiesNode.getKey().deduplicationEnabled = enableDeduplication; - - // Write back the new policies into zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), - jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - - log.info("[{}] Successfully {} on namespace {}/{}/{}", clientAppId(), - enableDeduplication ? "enabled" : "disabled", property, cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to modify deplication status for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn( - "[{}] Failed to modify deplication status on namespace {}/{}/{} expected policy node version={} : concurrent modification", - clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); - - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to modify deplication status on namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/bundles") - @ApiOperation(value = "Get the bundles split data.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") }) - public BundlesData getBundlesData(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - return policies.bundles; - } - - private BundlesData validateBundlesData(BundlesData initialBundles) { - SortedSet partitions = new TreeSet(); - for (String partition : initialBundles.getBoundaries()) { - Long partBoundary = Long.decode(partition); - partitions.add(String.format("0x%08x", partBoundary)); - } - if (partitions.size() != initialBundles.getBoundaries().size()) { - log.debug("Input bundles included repeated partition points. Ignored."); - } - try { - NamespaceBundleFactory.validateFullRange(partitions); - } catch (IllegalArgumentException iae) { - throw new RestException(Status.BAD_REQUEST, "Input bundles do not cover the whole hash range. first:" - + partitions.first() + ", last:" + partitions.last()); - } - List bundles = Lists.newArrayList(); - bundles.addAll(partitions); - return new BundlesData(bundles); - } - - private BundlesData getBundles(int numBundles) { - if (numBundles <= 0 || numBundles > MAX_BUNDLES) { - throw new RestException(Status.BAD_REQUEST, - "Invalid number of bundles. Number of numbles has to be in the range of (0, 2^32]."); - } - Long maxVal = ((long) 1) << 32; - Long segSize = maxVal / numBundles; - List partitions = Lists.newArrayList(); - partitions.add(String.format("0x%08x", 0l)); - Long curPartition = segSize; - for (int i = 0; i < numBundles; i++) { - if (i != numBundles - 1) { - partitions.add(String.format("0x%08x", curPartition)); - } else { - partitions.add(String.format("0x%08x", maxVal - 1)); - } - curPartition += segSize; - } - return new BundlesData(partitions); - } - - @PUT - @Path("/{property}/{cluster}/{namespace}/unload") - @ApiOperation(value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker" - + "removes all producers, consumers, and connections using this namespace, and close all destinations (including" - + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the" - + "broker completes the unloading action. This operation requires strictly super user privileges, since it would" - + "result in non-persistent message loss and unexpected connection closure to the clients.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 412, message = "Namespace is already unloaded or Namespace has bundles activated") }) - public void unloadNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - log.info("[{}] Unloading namespace {}/{}/{}", clientAppId(), property, cluster, namespace); - - validateSuperUserAccess(); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } else { - // check cluster ownership for a given global namespace: redirect if peer-cluster owns it - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - - Policies policies = getNamespacePolicies(property, cluster, namespace); - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - - List boundaries = policies.bundles.getBoundaries(); - for (int i = 0; i < boundaries.size() - 1; i++) { - String bundle = String.format("%s_%s", boundaries.get(i), boundaries.get(i + 1)); - try { - pulsar().getAdminClient().namespaces().unloadNamespaceBundle(nsName.toString(), bundle); - } catch (PulsarServerException | PulsarAdminException e) { - log.error(String.format("[%s] Failed to unload namespace %s/%s/%s", clientAppId(), property, cluster, - namespace), e); - throw new RestException(e); - } - } - log.info("[{}] Successfully unloaded all the bundles in namespace {}/{}/{}", clientAppId(), property, cluster, - namespace); - } - - @PUT - @Path("/{property}/{cluster}/{namespace}/{bundle}/unload") - @ApiOperation(value = "Unload a namespace bundle") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) - public void unloadNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - log.info("[{}] Unloading namespace bundle {}/{}/{}/{}", clientAppId(), property, cluster, namespace, - bundleRange); - - validateSuperUserAccess(); - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } else { - // check cluster ownership for a given global namespace: redirect if peer-cluster owns it - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - validatePoliciesReadOnlyAccess(); - - if (!isBundleOwnedByAnyBroker(fqnn, policies.bundles, bundleRange)) { - log.info("[{}] Namespace bundle is not owned by any broker {}/{}/{}/{}", clientAppId(), property, cluster, - namespace, bundleRange); - return; - } - - NamespaceBundle nsBundle = validateNamespaceBundleOwnership(fqnn, policies.bundles, bundleRange, authoritative, - true); - try { - pulsar().getNamespaceService().unloadNamespaceBundle(nsBundle); - log.info("[{}] Successfully unloaded namespace bundle {}", clientAppId(), nsBundle.toString()); - } catch (Exception e) { - log.error("[{}] Failed to unload namespace bundle {}/{}", clientAppId(), fqnn.toString(), bundleRange, e); - throw new RestException(e); - } - } - - @PUT - @Path("/{property}/{cluster}/{namespace}/{bundle}/split") - @ApiOperation(value = "Split a namespace bundle") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) - public void splitNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, - @QueryParam("unload") @DefaultValue("false") boolean unload) { - log.info("[{}] Split namespace bundle {}/{}/{}/{}", clientAppId(), property, cluster, namespace, bundleRange); - - validateSuperUserAccess(); - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } else { - // check cluster ownership for a given global namespace: redirect if peer-cluster owns it - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - validatePoliciesReadOnlyAccess(); - NamespaceBundle nsBundle = validateNamespaceBundleOwnership(fqnn, policies.bundles, bundleRange, authoritative, - true); - - try { - pulsar().getNamespaceService().splitAndOwnBundle(nsBundle, unload).get(); - log.info("[{}] Successfully split namespace bundle {}", clientAppId(), nsBundle.toString()); - } catch (IllegalArgumentException e) { - log.error("[{}] Failed to split namespace bundle {}/{} due to {}", clientAppId(), fqnn.toString(), - bundleRange, e.getMessage()); - throw new RestException(Status.PRECONDITION_FAILED, "Split bundle failed due to invalid request"); - } catch (Exception e) { - log.error("[{}] Failed to split namespace bundle {}/{}", clientAppId(), fqnn.toString(), bundleRange, e); - throw new RestException(e); - } - } - - @POST - @Path("/{property}/{cluster}/{namespace}/dispatchRate") - @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) - public void setDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, DispatchRate dispatchRate) { - log.info("[{}] Set namespace dispatch-rate {}/{}/{}/{}", clientAppId(), property, cluster, namespace, - dispatchRate); - validateSuperUserAccess(); - - Entry policiesNode = null; - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - - try { - final String path = path(POLICIES, property, cluster, namespace); - // Force to read the data s.t. the watch to the cache content is setup. - policiesNode = policiesCache().getWithStat(path) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); - policiesNode.getKey().clusterDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); - - // Write back the new policies into zookeeper - globalZk().setData(path, jsonMapper().writeValueAsBytes(policiesNode.getKey()), - policiesNode.getValue().getVersion()); - policiesCache().invalidate(path); - - log.info("[{}] Successfully updated the dispatchRate for cluster on namespace {}/{}/{}", clientAppId(), - property, cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update the dispatchRate for cluster on namespace {}/{}/{}: does not exist", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn( - "[{}] Failed to update the dispatchRate for cluster on namespace {}/{}/{} expected policy node version={} : concurrent modification", - clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); - - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to update the dispatchRate for cluster on namespace {}/{}/{}", clientAppId(), - property, cluster, namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/dispatchRate") - @ApiOperation(value = "Get dispatch-rate configured for the namespace, -1 represents not configured yet") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public DispatchRate getDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - Policies policies = getNamespacePolicies(property, cluster, namespace); - DispatchRate dispatchRate = policies.clusterDispatchRate.get(pulsar().getConfiguration().getClusterName()); - if (dispatchRate != null) { - return dispatchRate; - } else { - throw new RestException(Status.NOT_FOUND, - "Dispatch-rate is not configured for cluster " + pulsar().getConfiguration().getClusterName()); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/backlogQuotaMap") - @ApiOperation(value = "Get backlog quota map on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public Map getBacklogQuotaMap(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - return policies.backlog_quota_map; - } - - @POST - @Path("/{property}/{cluster}/{namespace}/backlogQuota") - @ApiOperation(value = " Set a backlog quota for all the destinations on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification"), - @ApiResponse(code = 412, message = "Specified backlog quota exceeds retention quota. Increase retention quota and retry request") }) - public void setBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, - BacklogQuota backlogQuota) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - if (backlogQuotaType == null) { - backlogQuotaType = BacklogQuotaType.destination_storage; - } - - try { - Stat nodeStat = new Stat(); - final String path = path(POLICIES, property, cluster, namespace); - byte[] content = globalZk().getData(path, null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - RetentionPolicies r = policies.retention_policies; - if (r != null) { - Policies p = new Policies(); - p.backlog_quota_map.put(backlogQuotaType, backlogQuota); - if (!checkQuotas(p, r)) { - log.warn( - "[{}] Failed to update backlog configuration for namespace {}/{}/{}: conflicts with retention quota", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.PRECONDITION_FAILED, - "Backlog Quota exceeds configured retention quota for namespace. Please increase retention quota and retry"); - } - } - policies.backlog_quota_map.put(backlogQuotaType, backlogQuota); - globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully updated backlog quota map: namespace={}/{}/{}, map={}", clientAppId(), property, - cluster, namespace, jsonMapper().writeValueAsString(policies.backlog_quota_map)); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (RestException pfe) { - throw pfe; - } catch (Exception e) { - log.error("[{}] Failed to update backlog quota map for namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - @DELETE - @Path("/{property}/{cluster}/{namespace}/backlogQuota") - @ApiOperation(value = "Remove a backlog quota policy from a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void removeBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, - @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - if (backlogQuotaType == null) { - backlogQuotaType = BacklogQuotaType.destination_storage; - } - - try { - Stat nodeStat = new Stat(); - final String path = path(POLICIES, property, cluster, namespace); - byte[] content = globalZk().getData(path, null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - policies.backlog_quota_map.remove(backlogQuotaType); - globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully removed backlog namespace={}/{}/{}, quota={}", clientAppId(), property, cluster, - namespace, backlogQuotaType); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to update backlog quota map for namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/retention") - @ApiOperation(value = "Get retention config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public RetentionPolicies getRetention(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - if (policies.retention_policies == null) { - return new RetentionPolicies(config().getDefaultRetentionTimeInMinutes(), - config().getDefaultRetentionSizeInMB()); - } else { - return policies.retention_policies; - } - } - - @POST - @Path("/{property}/{cluster}/{namespace}/retention") - @ApiOperation(value = " Set retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification"), - @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) - public void setRetention(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, RetentionPolicies retention) { - validatePoliciesReadOnlyAccess(); - - try { - Stat nodeStat = new Stat(); - final String path = path(POLICIES, property, cluster, namespace); - byte[] content = globalZk().getData(path, null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - if (!checkQuotas(policies, retention)) { - log.warn( - "[{}] Failed to update retention configuration for namespace {}/{}/{}: conflicts with backlog quota", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.PRECONDITION_FAILED, - "Retention Quota must exceed configured backlog quota for namespace."); - } - policies.retention_policies = retention; - globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully updated retention configuration: namespace={}/{}/{}, map={}", clientAppId(), - property, cluster, namespace, jsonMapper().writeValueAsString(policies.retention_policies)); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update retention configuration for namespace {}/{}/{}: does not exist", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to update retention configuration for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (RestException pfe) { - throw pfe; - } catch (Exception e) { - log.error("[{}] Failed to update retention configuration for namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - - } - - private boolean checkQuotas(Policies policies, RetentionPolicies retention) { - Map backlog_quota_map = policies.backlog_quota_map; - if (backlog_quota_map.isEmpty() || retention.getRetentionSizeInMB() == 0) { - return true; - } - BacklogQuota quota = backlog_quota_map.get(BacklogQuotaType.destination_storage); - if (quota == null) { - quota = pulsar().getBrokerService().getBacklogQuotaManager().getDefaultQuota(); - } - if (quota.getLimit() >= ((long) retention.getRetentionSizeInMB() * 1024 * 1024)) { - return false; - } - return true; - } - - @POST - @Path("/{property}/{cluster}/{namespace}/persistence") - @ApiOperation(value = "Set the persistence configuration for all the destinations on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification"), - @ApiResponse(code = 400, message = "Invalid persistence policies") }) - public void setPersistence(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, PersistencePolicies persistence) { - validatePoliciesReadOnlyAccess(); - validatePersistencePolicies(persistence); - - try { - Stat nodeStat = new Stat(); - final String path = path(POLICIES, property, cluster, namespace); - byte[] content = globalZk().getData(path, null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - policies.persistence = persistence; - globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully updated persistence configuration: namespace={}/{}/{}, map={}", clientAppId(), - property, cluster, namespace, jsonMapper().writeValueAsString(policies.persistence)); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update persistence configuration for namespace {}/{}/{}: does not exist", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to update persistence configuration for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to update persistence configuration for namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - private void validatePersistencePolicies(PersistencePolicies persistence) { - try { - checkNotNull(persistence); - final ServiceConfiguration config = pulsar().getConfiguration(); - checkArgument(persistence.getBookkeeperEnsemble() <= config.getManagedLedgerMaxEnsembleSize(), - "Bookkeeper-Ensemble must be <= %s", config.getManagedLedgerMaxEnsembleSize()); - checkArgument(persistence.getBookkeeperWriteQuorum() <= config.getManagedLedgerMaxWriteQuorum(), - "Bookkeeper-WriteQuorum must be <= %s", config.getManagedLedgerMaxWriteQuorum()); - checkArgument(persistence.getBookkeeperAckQuorum() <= config.getManagedLedgerMaxAckQuorum(), - "Bookkeeper-AckQuorum must be <= %s", config.getManagedLedgerMaxAckQuorum()); - checkArgument( - (persistence.getBookkeeperEnsemble() >= persistence.getBookkeeperWriteQuorum()) - && (persistence.getBookkeeperWriteQuorum() >= persistence.getBookkeeperAckQuorum()), - "Bookkeeper Ensemble (%s) >= WriteQuorum (%s) >= AckQuoru (%s)", persistence.getBookkeeperEnsemble(), - persistence.getBookkeeperWriteQuorum(), persistence.getBookkeeperAckQuorum()); - }catch(NullPointerException | IllegalArgumentException e) { - throw new RestException(Status.PRECONDITION_FAILED, e.getMessage()); - } - } - - @GET - @Path("/{property}/{cluster}/{namespace}/persistence") - @ApiOperation(value = "Get the persistence configuration for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public PersistencePolicies getPersistence(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - if (policies.persistence == null) { - return new PersistencePolicies(config().getManagedLedgerDefaultEnsembleSize(), - config().getManagedLedgerDefaultWriteQuorum(), config().getManagedLedgerDefaultAckQuorum(), 0.0d); - } else { - return policies.persistence; - } - } - - @POST - @Path("/{property}/{cluster}/{namespace}/clearBacklog") - @ApiOperation(value = "Clear backlog for all destinations on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateAdminAccessOnProperty(property); - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - try { - NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName); - Exception exception = null; - for (NamespaceBundle nsBundle : bundles.getBundles()) { - try { - // check if the bundle is owned by any broker, if not then there is no backlog on this bundle to - // clear - if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) { - // TODO: make this admin call asynchronous - pulsar().getAdminClient().namespaces().clearNamespaceBundleBacklog(nsName.toString(), - nsBundle.getBundleRange()); - } - } catch (Exception e) { - if (exception == null) { - exception = e; - } - } - } - if (exception != null) { - if (exception instanceof PulsarAdminException) { - throw new RestException((PulsarAdminException) exception); - } else { - throw new RestException(exception.getCause()); - } - } - } catch (WebApplicationException wae) { - throw wae; - } catch (Exception e) { - throw new RestException(e); - } - log.info("[{}] Successfully cleared backlog on all the bundles for namespace {}", clientAppId(), - nsName.toString()); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog") - @ApiOperation(value = "Clear backlog for all destinations on a namespace bundle.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public void clearNamespaceBundleBacklog(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } else { - // check cluster ownership for a given global namespace: redirect if peer-cluster owns it - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true); - - clearBacklog(nsName, bundleRange, null); - log.info("[{}] Successfully cleared backlog on namespace bundle {}/{}", clientAppId(), nsName.toString(), - bundleRange); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}") - @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public void clearNamespaceBacklogForSubscription(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("subscription") String subscription, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateAdminAccessOnProperty(property); - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - try { - NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName); - Exception exception = null; - for (NamespaceBundle nsBundle : bundles.getBundles()) { - try { - // check if the bundle is owned by any broker, if not then there is no backlog on this bundle to - // clear - if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) { - // TODO: make this admin call asynchronous - pulsar().getAdminClient().namespaces().clearNamespaceBundleBacklogForSubscription( - nsName.toString(), nsBundle.getBundleRange(), subscription); - } - } catch (Exception e) { - if (exception == null) { - exception = e; - } - } - } - if (exception != null) { - if (exception instanceof PulsarAdminException) { - throw new RestException((PulsarAdminException) exception); - } else { - throw new RestException(exception.getCause()); - } - } - } catch (WebApplicationException wae) { - throw wae; - } catch (Exception e) { - throw new RestException(e); - } - log.info("[{}] Successfully cleared backlog for subscription {} on all the bundles for namespace {}", - clientAppId(), subscription, nsName.toString()); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}") - @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace bundle.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("subscription") String subscription, @PathParam("bundle") String bundleRange, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } else { - // check cluster ownership for a given global namespace: redirect if peer-cluster owns it - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true); - - clearBacklog(nsName, bundleRange, subscription); - log.info("[{}] Successfully cleared backlog for subscription {} on namespace bundle {}/{}", clientAppId(), - subscription, nsName.toString(), bundleRange); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}") - @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateAdminAccessOnProperty(property); - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - try { - NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName); - Exception exception = null; - for (NamespaceBundle nsBundle : bundles.getBundles()) { - try { - // check if the bundle is owned by any broker, if not then there are no subscriptions - if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) { - // TODO: make this admin call asynchronous - pulsar().getAdminClient().namespaces().unsubscribeNamespaceBundle(nsName.toString(), - nsBundle.getBundleRange(), subscription); - } - } catch (Exception e) { - if (exception == null) { - exception = e; - } - } - } - if (exception != null) { - if (exception instanceof PulsarAdminException) { - throw new RestException((PulsarAdminException) exception); - } else { - throw new RestException(exception.getCause()); - } - } - } catch (WebApplicationException wae) { - throw wae; - } catch (Exception e) { - throw new RestException(e); - } - log.info("[{}] Successfully unsubscribed {} on all the bundles for namespace {}", clientAppId(), subscription, - nsName.toString()); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}") - @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace bundle.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public void unsubscribeNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, - @PathParam("bundle") String bundleRange, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateAdminAccessOnProperty(property); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } else { - // check cluster ownership for a given global namespace: redirect if peer-cluster owns it - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true); - - unsubscribe(nsName, bundleRange, subscription); - log.info("[{}] Successfully unsubscribed {} on namespace bundle {}/{}", clientAppId(), subscription, - nsName.toString(), bundleRange); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode") - @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void setSubscriptionAuthMode(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, SubscriptionAuthMode subscriptionAuthMode) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - if (subscriptionAuthMode == null) { - subscriptionAuthMode = SubscriptionAuthMode.None; - } - - try { - Stat nodeStat = new Stat(); - final String path = path(POLICIES, property, cluster, namespace); - byte[] content = globalZk().getData(path, null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - policies.subscription_auth_mode = subscriptionAuthMode; - globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - log.info("[{}] Successfully updated subscription auth mode: namespace={}/{}/{}, map={}", clientAppId(), property, - cluster, namespace, jsonMapper().writeValueAsString(policies.backlog_quota_map)); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to update subscription auth mode for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to update subscription auth mode for namespace {}/{}/{}: concurrent modification", - clientAppId(), property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (RestException pfe) { - throw pfe; - } catch (Exception e) { - log.error("[{}] Failed to update subscription auth mode for namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - private void clearBacklog(NamespaceName nsName, String bundleRange, String subscription) { - try { - List topicList = pulsar().getBrokerService() - .getAllTopicsFromNamespaceBundle(nsName.toString(), nsName.toString() + "/" + bundleRange); - - List> futures = Lists.newArrayList(); - if (subscription != null) { - if (subscription.startsWith(pulsar().getConfiguration().getReplicatorPrefix())) { - subscription = PersistentReplicator.getRemoteCluster(subscription); - } - for (Topic topic : topicList) { - if(topic instanceof PersistentTopic) { - futures.add(((PersistentTopic)topic).clearBacklog(subscription)); - } - } - } else { - for (Topic topic : topicList) { - if(topic instanceof PersistentTopic) { - futures.add(((PersistentTopic)topic).clearBacklog()); - } - } - } - - FutureUtil.waitForAll(futures).get(); - } catch (Exception e) { - log.error("[{}] Failed to clear backlog for namespace {}/{}, subscription: {}", clientAppId(), - nsName.toString(), bundleRange, subscription, e); - throw new RestException(e); - } - } - - private void unsubscribe(NamespaceName nsName, String bundleRange, String subscription) { - try { - List topicList = pulsar().getBrokerService() - .getAllTopicsFromNamespaceBundle(nsName.toString(), nsName.toString() + "/" + bundleRange); - List> futures = Lists.newArrayList(); - if (subscription.startsWith(pulsar().getConfiguration().getReplicatorPrefix())) { - throw new RestException(Status.PRECONDITION_FAILED, "Cannot unsubscribe a replication cursor"); - } else { - for (Topic topic : topicList) { - Subscription sub = topic.getSubscription(subscription); - if (sub != null) { - futures.add(sub.delete()); - } - } - } - - FutureUtil.waitForAll(futures).get(); - } catch (RestException re) { - throw re; - } catch (Exception e) { - log.error("[{}] Failed to unsubscribe {} for namespace {}/{}", clientAppId(), subscription, - nsName.toString(), bundleRange, e); - if (e.getCause() instanceof SubscriptionBusyException) { - throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers"); - } - throw new RestException(e.getCause()); - } - } - - /** - * It validates that peer-clusters can't coexist in replication-clusters - * - * @param clusterName: - * given cluster whose peer-clusters can't be present into replication-cluster list - * @param clusters: - * replication-cluster list - */ - private void validatePeerClusterConflict(String clusterName, Set replicationClusters) { - try { - ClusterData clusterData = clustersCache().get(path("clusters", clusterName)).orElseThrow( - () -> new RestException(Status.PRECONDITION_FAILED, "Invalid replication cluster " + clusterName)); - Set peerClusters = clusterData.getPeerClusterNames(); - if (peerClusters != null && !peerClusters.isEmpty()) { - SetView conflictPeerClusters = Sets.intersection(peerClusters, replicationClusters); - if (!conflictPeerClusters.isEmpty()) { - log.warn("[{}] {}'s peer cluster can't be part of replication clusters {}", clientAppId(), - clusterName, conflictPeerClusters); - throw new RestException(Status.CONFLICT, - String.format("%s's peer-clusters %s can't be part of replication-clusters %s", clusterName, - conflictPeerClusters, replicationClusters)); - } - } - } catch (RestException re) { - throw re; - } catch (Exception e) { - log.warn("[{}] Failed to get cluster-data for {}", clientAppId(), clusterName, e); - } - } - - @POST - @Path("/{property}/{cluster}/{namespace}/encryptionRequired") - @ApiOperation(value = "Message encryption is required or not for all topics in a namespace") - @ApiResponses(value = { - @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), - @ApiResponse(code = 409, message = "Concurrent modification"), - }) - public void modifyEncryptionRequired(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, boolean encryptionRequired) { - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - Entry policiesNode = null; - - try { - // Force to read the data s.t. the watch to the cache content is setup. - policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); - policiesNode.getKey().encryption_required = encryptionRequired; - - // Write back the new policies into zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), - jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - - log.info("[{}] Successfully {} on namespace {}/{}/{}", clientAppId(), - encryptionRequired ? "true" : "false", property, cluster, namespace); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to modify encryption required status for namespace {}/{}/{}: does not exist", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (KeeperException.BadVersionException e) { - log.warn( - "[{}] Failed to modify encryption required status on namespace {}/{}/{} expected policy node version={} : concurrent modification", - clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); - - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to modify encryption required status on namespace {}/{}/{}", clientAppId(), property, - cluster, namespace, e); - throw new RestException(e); - } - } - - private static final Logger log = LoggerFactory.getLogger(Namespaces.class); -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java similarity index 96% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/BrokerStats.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java index 8fdd3130791d4..549a0332ec79a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/BrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; +package org.apache.pulsar.broker.admin.impl; import java.io.OutputStream; import java.util.Collection; @@ -25,13 +25,12 @@ import javax.ws.rs.GET; import javax.ws.rs.Path; import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response.Status; import javax.ws.rs.core.StreamingOutput; import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.loadbalance.ResourceUnit; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; @@ -47,16 +46,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; -@Path("/broker-stats") -@Api(value = "/broker-stats", description = "Stats for broker", tags = "broker-stats") -@Produces(MediaType.APPLICATION_JSON) -public class BrokerStats extends AdminResource { - private static final Logger log = LoggerFactory.getLogger(BrokerStats.class); +public class BrokerStatsBase extends AdminResource { + private static final Logger log = LoggerFactory.getLogger(BrokerStatsBase.class); @GET @Path("/metrics") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Brokers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java similarity index 95% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Brokers.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index d4a538e5576ed..8098f97f8379b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Brokers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; +package org.apache.pulsar.broker.admin.impl; import static org.apache.pulsar.broker.service.BrokerService.BROKER_SERVICE_CONFIGURATION_PATH; @@ -28,14 +28,12 @@ import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response.Status; import org.apache.bookkeeper.util.ZkUtils; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.loadbalance.LoadManager; -import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; @@ -48,17 +46,13 @@ import com.google.common.collect.Maps; -import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; -@Path("/brokers") -@Api(value = "/brokers", description = "Brokers admin apis", tags = "brokers") -@Produces(MediaType.APPLICATION_JSON) -public class Brokers extends AdminResource { - private static final Logger LOG = LoggerFactory.getLogger(Brokers.class); +public class BrokersBase extends AdminResource { + private static final Logger LOG = LoggerFactory.getLogger(BrokersBase.class); private int serviceConfigZkVersion = -1; @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Clusters.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java similarity index 98% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Clusters.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index 612e2e254b069..733105a370f3e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Clusters.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; +package org.apache.pulsar.broker.admin.impl; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; @@ -35,11 +35,10 @@ import javax.ws.rs.PUT; import javax.ws.rs.Path; import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response.Status; import org.apache.bookkeeper.util.ZkUtils; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.NamedEntity; @@ -50,7 +49,6 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; @@ -60,17 +58,11 @@ import com.fasterxml.jackson.databind.JsonMappingException; import com.google.common.collect.Maps; -import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.FAILURE_DOMAIN; - -@Path("/clusters") -@Api(value = "/clusters", description = "Cluster admin apis", tags = "clusters") -@Produces(MediaType.APPLICATION_JSON) -public class Clusters extends AdminResource { +public class ClustersBase extends AdminResource { @GET @ApiOperation(value = "Get the list of all the Pulsar clusters.", response = String.class, responseContainer = "Set") @@ -623,6 +615,6 @@ private void validateBrokerExistsInOtherDomain(final String cluster, final Strin } } - private static final Logger log = LoggerFactory.getLogger(Clusters.class); + private static final Logger log = LoggerFactory.getLogger(ClustersBase.class); } 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 new file mode 100644 index 0000000000000..0dda3aed06c3d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -0,0 +1,1246 @@ +/** + * 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.impl; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; + +import java.net.URI; +import java.net.URL; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; +import javax.ws.rs.core.UriBuilder; + +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.BrokerServiceException.SubscriptionBusyException; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentReplicator; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.naming.DestinationName; +import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.NamespaceBundleFactory; +import org.apache.pulsar.common.naming.NamespaceBundles; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; +import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.PersistencePolicies; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.collect.Sets.SetView; + +public abstract class NamespacesBase extends AdminResource { + + private static final long MAX_BUNDLES = ((long) 1) << 32; + + protected List internalGetPropertyNamespaces(String property) { + validateAdminAccessOnProperty(property); + + try { + return getListOfNamespaces(property); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to get namespace list for propery: {} - Does not exist", clientAppId(), property); + throw new RestException(Status.NOT_FOUND, "Property does not exist"); + } catch (Exception e) { + log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e); + throw new RestException(e); + } + } + + protected void internalCreateNamespace(Policies policies) { + validatePoliciesReadOnlyAccess(); + validateAdminAccessOnProperty(namespaceName.getProperty()); + + validatePolicies(namespaceName, policies); + + try { + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + + zkCreateOptimistic(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies)); + log.info("[{}] Created namespace {}", clientAppId(), namespaceName); + } catch (KeeperException.NodeExistsException e) { + log.warn("[{}] Failed to create namespace {} - already exists", clientAppId(), namespaceName); + throw new RestException(Status.CONFLICT, "Namespace already exists"); + } catch (Exception e) { + log.error("[{}] Failed to create namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + + @SuppressWarnings("deprecation") + protected void internalDeleteNamespace(boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + // ensure that non-global namespace is directed to the correct cluster + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + } + + Entry policiesNode = null; + Policies policies = null; + + // ensure the local cluster is the only cluster for the global namespace configuration + try { + policiesNode = policiesCache().getWithStat(path(POLICIES, namespaceName.toString())).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist.")); + + policies = policiesNode.getKey(); + if (namespaceName.isGlobal()) { + if (policies.replication_clusters.size() > 1) { + // There are still more than one clusters configured for the global namespace + throw new RestException(Status.PRECONDITION_FAILED, "Cannot delete the global namespace " + + namespaceName + ". There are still more than one replication clusters configured."); + } + if (policies.replication_clusters.size() == 1 + && !policies.replication_clusters.contains(config().getClusterName())) { + // the only replication cluster is other cluster, redirect + String replCluster = policies.replication_clusters.get(0); + ClusterData replClusterData = clustersCache().get(AdminResource.path("clusters", replCluster)) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, + "Cluster " + replCluster + " does not exist")); + URL replClusterUrl; + if (!config().isTlsEnabled()) { + replClusterUrl = new URL(replClusterData.getServiceUrl()); + } else if (!replClusterData.getServiceUrlTls().isEmpty()) { + replClusterUrl = new URL(replClusterData.getServiceUrlTls()); + } else { + throw new RestException(Status.PRECONDITION_FAILED, + "The replication cluster does not provide TLS encrypted service"); + } + URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(replClusterUrl.getHost()) + .port(replClusterUrl.getPort()).replaceQueryParam("authoritative", false).build(); + if (log.isDebugEnabled()) { + log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, + replCluster); + } + throw new WebApplicationException(Response.temporaryRedirect(redirect).build()); + } + } + } catch (WebApplicationException wae) { + throw wae; + } catch (Exception e) { + throw new RestException(e); + } + + boolean isEmpty; + try { + isEmpty = pulsar().getNamespaceService().getListOfDestinations(namespaceName).isEmpty(); + } catch (Exception e) { + throw new RestException(e); + } + + if (!isEmpty) { + log.debug("Found destinations on namespace {}", namespaceName); + throw new RestException(Status.CONFLICT, "Cannot delete non empty namespace"); + } + + // set the policies to deleted so that somebody else cannot acquire this namespace + try { + policies.deleted = true; + globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies), + policiesNode.getValue().getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + } catch (Exception e) { + log.error("[{}] Failed to delete namespace on global ZK {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + + // remove from owned namespace map and ephemeral node from ZK + try { + NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory() + .getBundles(namespaceName); + for (NamespaceBundle bundle : bundles.getBundles()) { + // check if the bundle is owned by any broker, if not then we do not need to delete the bundle + if (pulsar().getNamespaceService().getOwner(bundle).isPresent()) { + pulsar().getAdminClient().namespaces().deleteNamespaceBundle(namespaceName.toString(), + bundle.getBundleRange()); + } + } + + // 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 lcaolZkPolicyPath = joinPath(LOCAL_POLICIES_ROOT, namespaceName.toString()); + globalZk().delete(globalZkPolicyPath, -1); + localZk().delete(lcaolZkPolicyPath, -1); + policiesCache().invalidate(globalZkPolicyPath); + localCacheService().policiesCache().invalidate(lcaolZkPolicyPath); + } catch (PulsarAdminException cae) { + throw new RestException(cae); + } catch (Exception e) { + log.error("[{}] Failed to remove owned namespace {}", clientAppId(), namespaceName, e); + // avoid throwing exception in case of the second failure + } + + } + + @SuppressWarnings("deprecation") + protected void internalDeleteNamespaceBundle(String bundleRange, boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + // ensure that non-global namespace is directed to the correct cluster + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + } + + Policies policies = getNamespacePolicies(namespaceName); + // ensure the local cluster is the only cluster for the global namespace configuration + try { + if (namespaceName.isGlobal()) { + if (policies.replication_clusters.size() > 1) { + // There are still more than one clusters configured for the global namespace + throw new RestException(Status.PRECONDITION_FAILED, "Cannot delete the global namespace " + + namespaceName + ". There are still more than one replication clusters configured."); + } + if (policies.replication_clusters.size() == 1 + && !policies.replication_clusters.contains(config().getClusterName())) { + // the only replication cluster is other cluster, redirect + String replCluster = policies.replication_clusters.get(0); + ClusterData replClusterData = clustersCache().get(AdminResource.path("clusters", replCluster)) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, + "Cluser " + replCluster + " does not exist")); + URL replClusterUrl; + if (!config().isTlsEnabled()) { + replClusterUrl = new URL(replClusterData.getServiceUrl()); + } else if (!replClusterData.getServiceUrlTls().isEmpty()) { + replClusterUrl = new URL(replClusterData.getServiceUrlTls()); + } else { + throw new RestException(Status.PRECONDITION_FAILED, + "The replication cluster does not provide TLS encrypted service"); + } + URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(replClusterUrl.getHost()) + .port(replClusterUrl.getPort()).replaceQueryParam("authoritative", false).build(); + log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, replCluster); + throw new WebApplicationException(Response.temporaryRedirect(redirect).build()); + } + } + } catch (WebApplicationException wae) { + throw wae; + } catch (Exception e) { + throw new RestException(e); + } + + NamespaceBundle bundle = validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange, + authoritative, true); + try { + List destinations = pulsar().getNamespaceService().getListOfDestinations(namespaceName); + for (String destination : destinations) { + NamespaceBundle destinationBundle = (NamespaceBundle) pulsar().getNamespaceService() + .getBundle(DestinationName.get(destination)); + if (bundle.equals(destinationBundle)) { + throw new RestException(Status.CONFLICT, "Cannot delete non empty bundle"); + } + } + + // remove from owned namespace map and ephemeral node from ZK + pulsar().getNamespaceService().removeOwnedServiceUnit(bundle); + } catch (WebApplicationException wae) { + throw wae; + } catch (Exception e) { + log.error("[{}] Failed to remove namespace bundle {}/{}", clientAppId(), namespaceName.toString(), + bundleRange, e); + throw new RestException(e); + } + } + + protected void internalGrantPermissionOnNamespace(String role, Set actions) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + try { + pulsar().getBrokerService().getAuthorizationService() + .grantPermissionAsync(namespaceName, actions, role, null/*additional auth-data json*/) + .get(); + } catch (InterruptedException e) { + log.error("[{}] Failed to get permissions for namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } catch (ExecutionException e) { + if (e.getCause() instanceof IllegalArgumentException) { + log.warn("[{}] Failed to set permissions for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } else if (e.getCause() instanceof IllegalStateException) { + log.warn("[{}] Failed to set permissions for namespace {}: concurrent modification", + clientAppId(), namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } else { + log.error("[{}] Failed to get permissions for namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + } + + protected void internalRevokePermissionsOnNamespace(String role) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + try { + Stat nodeStat = new Stat(); + byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + policies.auth_policies.namespace_auth.remove(role); + + // Write back the new policies into zookeeper + globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies), + nodeStat.getVersion()); + + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + log.info("[{}] Successfully revoked access for role {} - namespace {}", clientAppId(), role, namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to revoke permissions for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to revoke permissions on namespace {}: concurrent modification", clientAppId(), + namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to revoke permissions on namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + + protected List internalGetNamespaceReplicationClusters() { + if (!namespaceName.isGlobal()) { + throw new RestException(Status.PRECONDITION_FAILED, + "Cannot get the replication clusters for a non-global namespace"); + } + + Policies policies = getNamespacePolicies(namespaceName); + return policies.replication_clusters; + } + + protected void internalSetNamespaceReplicationClusters(List clusterIds) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + Set replicationClusterSet = Sets.newHashSet(clusterIds); + if (!namespaceName.isGlobal()) { + throw new RestException(Status.PRECONDITION_FAILED, "Cannot set replication on a non-global namespace"); + } + + if (replicationClusterSet.contains("global")) { + throw new RestException(Status.PRECONDITION_FAILED, + "Cannot specify global in the list of replication clusters"); + } + + Set clusters = clusters(); + for (String clusterId : replicationClusterSet) { + if (!clusters.contains(clusterId)) { + throw new RestException(Status.FORBIDDEN, "Invalid cluster id: " + clusterId); + } + validatePeerClusterConflict(clusterId, replicationClusterSet); + } + + for (String clusterId : replicationClusterSet) { + validateClusterForProperty(namespaceName.getProperty(), clusterId); + } + + Entry policiesNode = null; + + try { + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path(POLICIES, namespaceName.toString())).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist")); + policiesNode.getKey().replication_clusters = clusterIds; + + // Write back the new policies into zookeeper + globalZk().setData(path(POLICIES, namespaceName.toString()), + jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + + log.info("[{}] Successfully updated the replication clusters on namespace {}", clientAppId(), + namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update the replication clusters for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to update the replication clusters on namespace {} expected policy node version={} : concurrent modification", + clientAppId(), namespaceName, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update the replication clusters on namespace {}", clientAppId(), namespaceName, + e); + throw new RestException(e); + } + } + + protected void internalSetNamespaceMessageTTL(int messageTTL) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + if (messageTTL < 0) { + throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL"); + } + + Entry policiesNode = null; + + try { + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path(POLICIES, namespaceName.toString())).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist")); + policiesNode.getKey().message_ttl_in_seconds = messageTTL; + + // Write back the new policies into zookeeper + globalZk().setData(path(POLICIES, namespaceName.toString()), + jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + + log.info("[{}] Successfully updated the message TTL on namespace {}", clientAppId(), namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update the message TTL for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to update the message TTL on namespace {} expected policy node version={} : concurrent modification", + clientAppId(), namespaceName, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update the message TTL on namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + + protected void internalModifyDeduplication(boolean enableDeduplication) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + Entry policiesNode = null; + + try { + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path(POLICIES, namespaceName.toString())).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist")); + policiesNode.getKey().deduplicationEnabled = enableDeduplication; + + // Write back the new policies into zookeeper + globalZk().setData(path(POLICIES, namespaceName.toString()), + jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + + log.info("[{}] Successfully {} on namespace {}", clientAppId(), + enableDeduplication ? "enabled" : "disabled", namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to modify deplication status for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to modify deplication status on namespace {} expected policy node version={} : concurrent modification", + clientAppId(), namespaceName, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to modify deplication status on namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + + @SuppressWarnings("deprecation") + protected void internalUnloadNamespace() { + log.info("[{}] Unloading namespace {}", clientAppId()); + + validateSuperUserAccess(); + + if (namespaceName.isGlobal()) { + // check cluster ownership for a given global namespace: redirect if peer-cluster owns it + validateGlobalNamespaceOwnership(namespaceName); + } else { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + Policies policies = getNamespacePolicies(namespaceName); + + List boundaries = policies.bundles.getBoundaries(); + for (int i = 0; i < boundaries.size() - 1; i++) { + String bundle = String.format("%s_%s", boundaries.get(i), boundaries.get(i + 1)); + try { + pulsar().getAdminClient().namespaces().unloadNamespaceBundle(namespaceName.toString(), bundle); + } catch (PulsarServerException | PulsarAdminException e) { + log.error(String.format("[%s] Failed to unload namespace %s", clientAppId(), namespaceName), e); + throw new RestException(e); + } + } + + log.info("[{}] Successfully unloaded all the bundles in namespace {}/{}/{}", clientAppId(), namespaceName); + } + + @SuppressWarnings("deprecation") + public void internalUnloadNamespaceBundle(String bundleRange, boolean authoritative) { + log.info("[{}] Unloading namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange); + + validateSuperUserAccess(); + Policies policies = getNamespacePolicies(namespaceName); + + if (namespaceName.isGlobal()) { + // check cluster ownership for a given global namespace: redirect if peer-cluster owns it + validateGlobalNamespaceOwnership(namespaceName); + } else { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + validatePoliciesReadOnlyAccess(); + + if (!isBundleOwnedByAnyBroker(namespaceName, policies.bundles, bundleRange)) { + log.info("[{}] Namespace bundle is not owned by any broker {}/{}", clientAppId(), namespaceName, + bundleRange); + return; + } + + NamespaceBundle nsBundle = validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange, + authoritative, true); + try { + pulsar().getNamespaceService().unloadNamespaceBundle(nsBundle); + log.info("[{}] Successfully unloaded namespace bundle {}", clientAppId(), nsBundle.toString()); + } catch (Exception e) { + log.error("[{}] Failed to unload namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, e); + throw new RestException(e); + } + } + + @SuppressWarnings("deprecation") + protected void internalSplitNamespaceBundle(String bundleRange, boolean authoritative, boolean unload) { + log.info("[{}] Split namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange); + + validateSuperUserAccess(); + Policies policies = getNamespacePolicies(namespaceName); + + if (namespaceName.isGlobal()) { + // check cluster ownership for a given global namespace: redirect if peer-cluster owns it + validateGlobalNamespaceOwnership(namespaceName); + } else { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + validatePoliciesReadOnlyAccess(); + NamespaceBundle nsBundle = validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange, + authoritative, true); + + try { + pulsar().getNamespaceService().splitAndOwnBundle(nsBundle, unload).get(); + log.info("[{}] Successfully split namespace bundle {}", clientAppId(), nsBundle.toString()); + } catch (IllegalArgumentException e) { + log.error("[{}] Failed to split namespace bundle {}/{} due to {}", clientAppId(), namespaceName, + bundleRange, e.getMessage()); + throw new RestException(Status.PRECONDITION_FAILED, "Split bundle failed due to invalid request"); + } catch (Exception e) { + log.error("[{}] Failed to split namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, e); + throw new RestException(e); + } + } + + protected void internalSetDispatchRate(DispatchRate dispatchRate) { + log.info("[{}] Set namespace dispatch-rate {}/{}", clientAppId(), namespaceName, dispatchRate); + validateSuperUserAccess(); + + Entry policiesNode = null; + + try { + final String path = path(POLICIES, namespaceName.toString()); + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist")); + policiesNode.getKey().clusterDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate); + + // Write back the new policies into zookeeper + globalZk().setData(path, jsonMapper().writeValueAsBytes(policiesNode.getKey()), + policiesNode.getValue().getVersion()); + policiesCache().invalidate(path); + + log.info("[{}] Successfully updated the dispatchRate for cluster on namespace {}", clientAppId(), + namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update the dispatchRate for cluster on namespace {}: does not exist", + clientAppId(), namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to update the dispatchRate for cluster on namespace {} expected policy node version={} : concurrent modification", + clientAppId(), namespaceName, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update the dispatchRate for cluster on namespace {}", clientAppId(), + namespaceName, e); + throw new RestException(e); + } + } + + protected DispatchRate internalGetDispatchRate() { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + Policies policies = getNamespacePolicies(namespaceName); + DispatchRate dispatchRate = policies.clusterDispatchRate.get(pulsar().getConfiguration().getClusterName()); + if (dispatchRate != null) { + return dispatchRate; + } else { + throw new RestException(Status.NOT_FOUND, + "Dispatch-rate is not configured for cluster " + pulsar().getConfiguration().getClusterName()); + } + } + + protected void internalSetBacklogQuota(BacklogQuotaType backlogQuotaType, BacklogQuota backlogQuota) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + if (backlogQuotaType == null) { + backlogQuotaType = BacklogQuotaType.destination_storage; + } + + try { + Stat nodeStat = new Stat(); + final String path = path(POLICIES, namespaceName.toString()); + byte[] content = globalZk().getData(path, null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + RetentionPolicies r = policies.retention_policies; + if (r != null) { + Policies p = new Policies(); + p.backlog_quota_map.put(backlogQuotaType, backlogQuota); + if (!checkQuotas(p, r)) { + log.warn( + "[{}] Failed to update backlog configuration for namespace {}: conflicts with retention quota", + clientAppId(), namespaceName); + throw new RestException(Status.PRECONDITION_FAILED, + "Backlog Quota exceeds configured retention quota for namespace. Please increase retention quota and retry"); + } + } + policies.backlog_quota_map.put(backlogQuotaType, backlogQuota); + globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + log.info("[{}] Successfully updated backlog quota map: namespace={}, map={}", clientAppId(), namespaceName, + jsonMapper().writeValueAsString(policies.backlog_quota_map)); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update backlog quota map for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to update backlog quota map for namespace {}: concurrent modification", clientAppId(), + namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (RestException pfe) { + throw pfe; + } catch (Exception e) { + log.error("[{}] Failed to update backlog quota map for namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + + protected void internalRemoveBacklogQuota(BacklogQuotaType backlogQuotaType) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + if (backlogQuotaType == null) { + backlogQuotaType = BacklogQuotaType.destination_storage; + } + + try { + Stat nodeStat = new Stat(); + final String path = path(POLICIES, namespaceName.toString()); + byte[] content = globalZk().getData(path, null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + policies.backlog_quota_map.remove(backlogQuotaType); + globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + log.info("[{}] Successfully removed backlog namespace={}, quota={}", clientAppId(), namespaceName, + backlogQuotaType); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update backlog quota map for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to update backlog quota map for namespace {}: concurrent modification", clientAppId(), + namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update backlog quota map for namespace {}", clientAppId(), namespaceName, e); + throw new RestException(e); + } + } + + protected void internalSetRetention(RetentionPolicies retention) { + validatePoliciesReadOnlyAccess(); + + try { + Stat nodeStat = new Stat(); + final String path = path(POLICIES, namespaceName.toString()); + byte[] content = globalZk().getData(path, null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + if (!checkQuotas(policies, retention)) { + log.warn("[{}] Failed to update retention configuration for namespace {}: conflicts with backlog quota", + clientAppId(), namespaceName); + throw new RestException(Status.PRECONDITION_FAILED, + "Retention Quota must exceed configured backlog quota for namespace."); + } + policies.retention_policies = retention; + globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + log.info("[{}] Successfully updated retention configuration: namespace={}, map={}", clientAppId(), + namespaceName, jsonMapper().writeValueAsString(policies.retention_policies)); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update retention configuration for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to update retention configuration for namespace {}: concurrent modification", + clientAppId(), namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (RestException pfe) { + throw pfe; + } catch (Exception e) { + log.error("[{}] Failed to update retention configuration for namespace {}", clientAppId(), namespaceName, + e); + throw new RestException(e); + } + } + + protected void internalSetPersistence(PersistencePolicies persistence) { + validatePoliciesReadOnlyAccess(); + validatePersistencePolicies(persistence); + + try { + Stat nodeStat = new Stat(); + final String path = path(POLICIES, namespaceName.toString()); + byte[] content = globalZk().getData(path, null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + policies.persistence = persistence; + globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + log.info("[{}] Successfully updated persistence configuration: namespace={}, map={}", clientAppId(), + namespaceName, jsonMapper().writeValueAsString(policies.persistence)); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update persistence configuration for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to update persistence configuration for namespace {}: concurrent modification", + clientAppId(), namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update persistence configuration for namespace {}", clientAppId(), namespaceName, + e); + throw new RestException(e); + } + } + + protected PersistencePolicies internalGetPersistence() { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + Policies policies = getNamespacePolicies(namespaceName); + if (policies.persistence == null) { + return new PersistencePolicies(config().getManagedLedgerDefaultEnsembleSize(), + config().getManagedLedgerDefaultWriteQuorum(), config().getManagedLedgerDefaultAckQuorum(), 0.0d); + } else { + return policies.persistence; + } + } + + protected void internalClearNamespaceBacklog(boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + try { + NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory() + .getBundles(namespaceName); + Exception exception = null; + for (NamespaceBundle nsBundle : bundles.getBundles()) { + try { + // check if the bundle is owned by any broker, if not then there is no backlog on this bundle to + // clear + if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) { + // TODO: make this admin call asynchronous + pulsar().getAdminClient().namespaces().clearNamespaceBundleBacklog(namespaceName.toString(), + nsBundle.getBundleRange()); + } + } catch (Exception e) { + if (exception == null) { + exception = e; + } + } + } + if (exception != null) { + if (exception instanceof PulsarAdminException) { + throw new RestException((PulsarAdminException) exception); + } else { + throw new RestException(exception.getCause()); + } + } + } catch (WebApplicationException wae) { + throw wae; + } catch (Exception e) { + throw new RestException(e); + } + log.info("[{}] Successfully cleared backlog on all the bundles for namespace {}", clientAppId(), namespaceName); + } + + @SuppressWarnings("deprecation") + protected void internalClearNamespaceBundleBacklog(String bundleRange, boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + Policies policies = getNamespacePolicies(namespaceName); + + if (namespaceName.isGlobal()) { + // check cluster ownership for a given global namespace: redirect if peer-cluster owns it + validateGlobalNamespaceOwnership(namespaceName); + } else { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange, authoritative, true); + + clearBacklog(namespaceName, bundleRange, null); + log.info("[{}] Successfully cleared backlog on namespace bundle {}/{}", clientAppId(), namespaceName, + bundleRange); + } + + protected void internalClearNamespaceBacklogForSubscription(String subscription, boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + try { + NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory() + .getBundles(namespaceName); + Exception exception = null; + for (NamespaceBundle nsBundle : bundles.getBundles()) { + try { + // check if the bundle is owned by any broker, if not then there is no backlog on this bundle to + // clear + if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) { + // TODO: make this admin call asynchronous + pulsar().getAdminClient().namespaces().clearNamespaceBundleBacklogForSubscription( + namespaceName.toString(), nsBundle.getBundleRange(), subscription); + } + } catch (Exception e) { + if (exception == null) { + exception = e; + } + } + } + if (exception != null) { + if (exception instanceof PulsarAdminException) { + throw new RestException((PulsarAdminException) exception); + } else { + throw new RestException(exception.getCause()); + } + } + } catch (WebApplicationException wae) { + throw wae; + } catch (Exception e) { + throw new RestException(e); + } + log.info("[{}] Successfully cleared backlog for subscription {} on all the bundles for namespace {}", + clientAppId(), subscription, namespaceName); + } + + @SuppressWarnings("deprecation") + protected void internalClearNamespaceBundleBacklogForSubscription(String subscription, String bundleRange, + boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + Policies policies = getNamespacePolicies(namespaceName); + + if (namespaceName.isGlobal()) { + // check cluster ownership for a given global namespace: redirect if peer-cluster owns it + validateGlobalNamespaceOwnership(namespaceName); + } else { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange, authoritative, true); + + clearBacklog(namespaceName, bundleRange, subscription); + log.info("[{}] Successfully cleared backlog for subscription {} on namespace bundle {}/{}", clientAppId(), + subscription, namespaceName, bundleRange); + } + + protected void internalUnsubscribeNamespace(String subscription, boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + try { + NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory() + .getBundles(namespaceName); + Exception exception = null; + for (NamespaceBundle nsBundle : bundles.getBundles()) { + try { + // check if the bundle is owned by any broker, if not then there are no subscriptions + if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) { + // TODO: make this admin call asynchronous + pulsar().getAdminClient().namespaces().unsubscribeNamespaceBundle(namespaceName.toString(), + nsBundle.getBundleRange(), subscription); + } + } catch (Exception e) { + if (exception == null) { + exception = e; + } + } + } + if (exception != null) { + if (exception instanceof PulsarAdminException) { + throw new RestException((PulsarAdminException) exception); + } else { + throw new RestException(exception.getCause()); + } + } + } catch (WebApplicationException wae) { + throw wae; + } catch (Exception e) { + throw new RestException(e); + } + log.info("[{}] Successfully unsubscribed {} on all the bundles for namespace {}", clientAppId(), subscription, + namespaceName); + } + + @SuppressWarnings("deprecation") + protected void internalUnsubscribeNamespaceBundle(String subscription, String bundleRange, boolean authoritative) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + Policies policies = getNamespacePolicies(namespaceName); + + if (namespaceName.isGlobal()) { + // check cluster ownership for a given global namespace: redirect if peer-cluster owns it + validateGlobalNamespaceOwnership(namespaceName); + } else { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange, authoritative, true); + + unsubscribe(namespaceName, bundleRange, subscription); + log.info("[{}] Successfully unsubscribed {} on namespace bundle {}/{}", clientAppId(), subscription, + namespaceName, bundleRange); + } + + protected void internalSetSubscriptionAuthMode(SubscriptionAuthMode subscriptionAuthMode) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + if (subscriptionAuthMode == null) { + subscriptionAuthMode = SubscriptionAuthMode.None; + } + + try { + Stat nodeStat = new Stat(); + final String path = path(POLICIES, namespaceName.toString()); + byte[] content = globalZk().getData(path, null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + policies.subscription_auth_mode = subscriptionAuthMode; + globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + log.info("[{}] Successfully updated subscription auth mode: namespace={}, map={}", clientAppId(), + namespaceName, jsonMapper().writeValueAsString(policies.backlog_quota_map)); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update subscription auth mode for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to update subscription auth mode for namespace {}/{}/{}: concurrent modification", + clientAppId(), namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (RestException pfe) { + throw pfe; + } catch (Exception e) { + log.error("[{}] Failed to update subscription auth mode for namespace {}/{}/{}", clientAppId(), + namespaceName, e); + throw new RestException(e); + } + } + + protected void internalModifyEncryptionRequired(boolean encryptionRequired) { + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + Entry policiesNode = null; + + try { + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path(POLICIES, namespaceName.toString())).orElseThrow( + () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist")); + policiesNode.getKey().encryption_required = encryptionRequired; + + // Write back the new policies into zookeeper + globalZk().setData(path(POLICIES, namespaceName.toString()), + jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + + log.info("[{}] Successfully {} on namespace {}", clientAppId(), encryptionRequired ? "true" : "false", + namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to modify encryption required status for namespace {}: does not exist", clientAppId(), + namespaceName); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to modify encryption required status on namespace {} expected policy node version={} : concurrent modification", + clientAppId(), namespaceName, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to modify encryption required status on namespace {}", clientAppId(), namespaceName, + e); + throw new RestException(e); + } + } + + private void validatePersistencePolicies(PersistencePolicies persistence) { + try { + checkNotNull(persistence); + final ServiceConfiguration config = pulsar().getConfiguration(); + checkArgument(persistence.getBookkeeperEnsemble() <= config.getManagedLedgerMaxEnsembleSize(), + "Bookkeeper-Ensemble must be <= %s", config.getManagedLedgerMaxEnsembleSize()); + checkArgument(persistence.getBookkeeperWriteQuorum() <= config.getManagedLedgerMaxWriteQuorum(), + "Bookkeeper-WriteQuorum must be <= %s", config.getManagedLedgerMaxWriteQuorum()); + checkArgument(persistence.getBookkeeperAckQuorum() <= config.getManagedLedgerMaxAckQuorum(), + "Bookkeeper-AckQuorum must be <= %s", config.getManagedLedgerMaxAckQuorum()); + checkArgument( + (persistence.getBookkeeperEnsemble() >= persistence.getBookkeeperWriteQuorum()) + && (persistence.getBookkeeperWriteQuorum() >= persistence.getBookkeeperAckQuorum()), + "Bookkeeper Ensemble (%s) >= WriteQuorum (%s) >= AckQuoru (%s)", + persistence.getBookkeeperEnsemble(), persistence.getBookkeeperWriteQuorum(), + persistence.getBookkeeperAckQuorum()); + } catch (NullPointerException | IllegalArgumentException e) { + throw new RestException(Status.PRECONDITION_FAILED, e.getMessage()); + } + } + + protected RetentionPolicies internalGetRetention() { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + Policies policies = getNamespacePolicies(namespaceName); + if (policies.retention_policies == null) { + return new RetentionPolicies(config().getDefaultRetentionTimeInMinutes(), + config().getDefaultRetentionSizeInMB()); + } else { + return policies.retention_policies; + } + } + + private boolean checkQuotas(Policies policies, RetentionPolicies retention) { + Map backlog_quota_map = policies.backlog_quota_map; + if (backlog_quota_map.isEmpty() || retention.getRetentionSizeInMB() == 0) { + return true; + } + BacklogQuota quota = backlog_quota_map.get(BacklogQuotaType.destination_storage); + if (quota == null) { + quota = pulsar().getBrokerService().getBacklogQuotaManager().getDefaultQuota(); + } + if (quota.getLimit() >= ((long) retention.getRetentionSizeInMB() * 1024 * 1024)) { + return false; + } + return true; + } + + private void clearBacklog(NamespaceName nsName, String bundleRange, String subscription) { + try { + List topicList = pulsar().getBrokerService().getAllTopicsFromNamespaceBundle(nsName.toString(), + nsName.toString() + "/" + bundleRange); + + List> futures = Lists.newArrayList(); + if (subscription != null) { + if (subscription.startsWith(pulsar().getConfiguration().getReplicatorPrefix())) { + subscription = PersistentReplicator.getRemoteCluster(subscription); + } + for (Topic topic : topicList) { + if (topic instanceof PersistentTopic) { + futures.add(((PersistentTopic) topic).clearBacklog(subscription)); + } + } + } else { + for (Topic topic : topicList) { + if (topic instanceof PersistentTopic) { + futures.add(((PersistentTopic) topic).clearBacklog()); + } + } + } + + FutureUtil.waitForAll(futures).get(); + } catch (Exception e) { + log.error("[{}] Failed to clear backlog for namespace {}/{}, subscription: {}", clientAppId(), + nsName.toString(), bundleRange, subscription, e); + throw new RestException(e); + } + } + + private void unsubscribe(NamespaceName nsName, String bundleRange, String subscription) { + try { + List topicList = pulsar().getBrokerService().getAllTopicsFromNamespaceBundle(nsName.toString(), + nsName.toString() + "/" + bundleRange); + List> futures = Lists.newArrayList(); + if (subscription.startsWith(pulsar().getConfiguration().getReplicatorPrefix())) { + throw new RestException(Status.PRECONDITION_FAILED, "Cannot unsubscribe a replication cursor"); + } else { + for (Topic topic : topicList) { + Subscription sub = topic.getSubscription(subscription); + if (sub != null) { + futures.add(sub.delete()); + } + } + } + + FutureUtil.waitForAll(futures).get(); + } catch (RestException re) { + throw re; + } catch (Exception e) { + log.error("[{}] Failed to unsubscribe {} for namespace {}/{}", clientAppId(), subscription, + nsName.toString(), bundleRange, e); + if (e.getCause() instanceof SubscriptionBusyException) { + throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers"); + } + throw new RestException(e.getCause()); + } + } + + /** + * It validates that peer-clusters can't coexist in replication-clusters + * + * @param clusterName: + * given cluster whose peer-clusters can't be present into replication-cluster list + * @param clusters: + * replication-cluster list + */ + private void validatePeerClusterConflict(String clusterName, Set replicationClusters) { + try { + ClusterData clusterData = clustersCache().get(path("clusters", clusterName)).orElseThrow( + () -> new RestException(Status.PRECONDITION_FAILED, "Invalid replication cluster " + clusterName)); + Set peerClusters = clusterData.getPeerClusterNames(); + if (peerClusters != null && !peerClusters.isEmpty()) { + SetView conflictPeerClusters = Sets.intersection(peerClusters, replicationClusters); + if (!conflictPeerClusters.isEmpty()) { + log.warn("[{}] {}'s peer cluster can't be part of replication clusters {}", clientAppId(), + clusterName, conflictPeerClusters); + throw new RestException(Status.CONFLICT, + String.format("%s's peer-clusters %s can't be part of replication-clusters %s", clusterName, + conflictPeerClusters, replicationClusters)); + } + } + } catch (RestException re) { + throw re; + } catch (Exception e) { + log.warn("[{}] Failed to get cluster-data for {}", clientAppId(), clusterName, e); + } + } + + protected BundlesData validateBundlesData(BundlesData initialBundles) { + SortedSet partitions = new TreeSet(); + for (String partition : initialBundles.getBoundaries()) { + Long partBoundary = Long.decode(partition); + partitions.add(String.format("0x%08x", partBoundary)); + } + if (partitions.size() != initialBundles.getBoundaries().size()) { + log.debug("Input bundles included repeated partition points. Ignored."); + } + try { + NamespaceBundleFactory.validateFullRange(partitions); + } catch (IllegalArgumentException iae) { + throw new RestException(Status.BAD_REQUEST, "Input bundles do not cover the whole hash range. first:" + + partitions.first() + ", last:" + partitions.last()); + } + List bundles = Lists.newArrayList(); + bundles.addAll(partitions); + return new BundlesData(bundles); + } + + protected BundlesData getBundles(int numBundles) { + if (numBundles <= 0 || numBundles > MAX_BUNDLES) { + throw new RestException(Status.BAD_REQUEST, + "Invalid number of bundles. Number of numbles has to be in the range of (0, 2^32]."); + } + Long maxVal = ((long) 1) << 32; + Long segSize = maxVal / numBundles; + List partitions = Lists.newArrayList(); + partitions.add(String.format("0x%08x", 0l)); + Long curPartition = segSize; + for (int i = 0; i < numBundles; i++) { + if (i != numBundles - 1) { + partitions.add(String.format("0x%08x", curPartition)); + } else { + partitions.add(String.format("0x%08x", maxVal - 1)); + } + curPartition += segSize; + } + return new BundlesData(partitions); + } + + private void validatePolicies(NamespaceName ns, Policies policies) { + // Validate cluster names and permissions + policies.replication_clusters.forEach(cluster -> validateClusterForProperty(ns.getProperty(), cluster)); + + if (policies.message_ttl_in_seconds < 0) { + throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL"); + } + + if (policies.bundles != null && policies.bundles.getNumBundles() > 0) { + if (policies.bundles.getBoundaries() == null || policies.bundles.getBoundaries().size() == 0) { + policies.bundles = getBundles(policies.bundles.getNumBundles()); + } else { + policies.bundles = validateBundlesData(policies.bundles); + } + } else { + int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles(); + policies.bundles = getBundles(defaultNumberOfBundles); + } + + if (policies.persistence != null) { + validatePersistencePolicies(policies.persistence); + } + } + + private static final Logger log = LoggerFactory.getLogger(NamespacesBase.class); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java similarity index 57% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 484a8735857b4..b459011d01322 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; +package org.apache.pulsar.broker.admin.impl; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; @@ -35,20 +35,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import javax.ws.rs.DELETE; -import javax.ws.rs.DefaultValue; -import javax.ws.rs.Encoded; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.PUT; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; import javax.ws.rs.WebApplicationException; import javax.ws.rs.container.AsyncResponse; -import javax.ws.rs.container.Suspended; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.ResponseBuilder; import javax.ws.rs.core.Response.Status; @@ -65,6 +53,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; @@ -90,7 +79,6 @@ import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.naming.DestinationDomain; import org.apache.pulsar.common.naming.DestinationName; -import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.AuthPolicies; @@ -113,60 +101,44 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.PooledByteBufAllocator; -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiOperation; -import io.swagger.annotations.ApiResponse; -import io.swagger.annotations.ApiResponses; /** */ -@Path("/persistent") -@Produces(MediaType.APPLICATION_JSON) -@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic") -public class PersistentTopics extends AdminResource { - private static final Logger log = LoggerFactory.getLogger(PersistentTopics.class); +public class PersistentTopicsBase extends AdminResource { + private static final Logger log = LoggerFactory.getLogger(PersistentTopicsBase.class); protected static final int PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS = 1000; private static final int OFFLINE_TOPIC_STAT_TTL_MINS = 10; private static final String DEPRECATED_CLIENT_VERSION_PREFIX = "Pulsar-CPP-v"; - private static final Version LEAST_SUPPORTED_CLIENT_VERSION_PREFIX = Version.forIntegers(1,21); + private static final Version LEAST_SUPPORTED_CLIENT_VERSION_PREFIX = Version.forIntegers(1, 21); - @GET - @Path("/{property}/{cluster}/{namespace}") - @ApiOperation(value = "Get the list of destinations under a namespace.", response = String.class, responseContainer = "List") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace doesn't exist") }) - public List getList(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); + protected List internalGetList() { + validateAdminAccessOnProperty(namespaceName.getProperty()); // Validate that namespace exists, throws 404 if it doesn't exist try { - policiesCache().get(path(POLICIES, property, cluster, namespace)); + policiesCache().get(path(POLICIES, namespaceName.toString())); } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get topic list {}/{}/{}: Namespace does not exist", clientAppId(), property, - cluster, namespace); + log.warn("[{}] Failed to get topic list {}: Namespace does not exist", clientAppId(), namespaceName); throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); } catch (Exception e) { - log.error("[{}] Failed to get topic list {}/{}/{}", clientAppId(), property, cluster, namespace, e); + log.error("[{}] Failed to get topic list {}", clientAppId(), namespaceName, e); throw new RestException(e); } List destinations = Lists.newArrayList(); try { - String path = String.format("/managed-ledgers/%s/%s/%s/%s", property, cluster, namespace, domain()); + String path = String.format("/managed-ledgers/%s/%s", namespaceName.toString(), domain()); for (String destination : managedLedgerListCache().get(path)) { if (domain().equals(DestinationDomain.persistent.toString())) { - destinations.add(DestinationName - .get(domain(), property, cluster, namespace, decode(destination)).toString()); + destinations.add(DestinationName.get(domain(), namespaceName, decode(destination)).toString()); } } } catch (KeeperException.NoNodeException e) { // NoNode means there are no destination in this domain for this namespace } catch (Exception e) { - log.error("[{}] Failed to get destination list for namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); + log.error("[{}] Failed to get destination list for namespace {}", clientAppId(), namespaceName, e); throw new RestException(e); } @@ -174,39 +146,34 @@ public List getList(@PathParam("property") String property, @PathParam(" return destinations; } - @GET - @Path("/{property}/{cluster}/{namespace}/partitioned") - @ApiOperation(value = "Get the list of partitioned topics under a namespace.", response = String.class, responseContainer = "List") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace doesn't exist") }) - public List getPartitionedTopicList(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); + protected List internalGetPartitionedTopicList() { + validateAdminAccessOnProperty(namespaceName.getProperty()); // Validate that namespace exists, throws 404 if it doesn't exist try { - policiesCache().get(path(POLICIES, property, cluster, namespace)); + policiesCache().get(path(POLICIES, namespaceName.toString())); } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get partitioned topic list {}/{}/{}: Namespace does not exist", clientAppId(), property, - cluster, namespace); + log.warn("[{}] Failed to get partitioned topic list {}: Namespace does not exist", clientAppId(), + namespaceName); throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); } catch (Exception e) { - log.error("[{}] Failed to get partitioned topic list for namespace {}/{}/{}", clientAppId(), property, cluster, namespace, e); + log.error("[{}] Failed to get partitioned topic list for namespace {}", clientAppId(), namespaceName, e); throw new RestException(e); } List partitionedTopics = Lists.newArrayList(); try { - String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain()); + String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain()); List destinations = globalZk().getChildren(partitionedTopicPath, false); - partitionedTopics = destinations.stream().map(s -> String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, decode(s))).collect( - Collectors.toList()); + partitionedTopics = destinations.stream() + .map(s -> String.format("persistent://%s/%s", namespaceName.toString(), decode(s))) + .collect(Collectors.toList()); } catch (KeeperException.NoNodeException e) { // NoNode means there are no partitioned topics in this domain for this namespace } catch (Exception e) { - log.error("[{}] Failed to get partitioned topic list for namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); + log.error("[{}] Failed to get partitioned topic list for namespace {}", clientAppId(), + namespaceName.toString(), e); throw new RestException(e); } @@ -214,23 +181,14 @@ public List getPartitionedTopicList(@PathParam("property") String proper return partitionedTopics; } - @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/permissions") - @ApiOperation(value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the" - + "namespace level combined (union) with any eventual specific permission set on the destination.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace doesn't exist") }) - public Map> getPermissionsOnDestination(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination) { + protected Map> internalGetPermissionsOnDestination() { // This operation should be reading from zookeeper and it should be allowed without having admin privileges - destination = decode(destination); - validateAdminAccessOnProperty(property); + validateAdminAccessOnProperty(namespaceName.getProperty()); - String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString(); + String destinationUri = destinationName.toString(); try { - Policies policies = policiesCache().get(path(POLICIES, property, cluster, namespace)) + Policies policies = policiesCache().get(path(POLICIES, namespaceName.toString())) .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); Map> permissions = Maps.newTreeMap(); @@ -264,47 +222,38 @@ public Map> getPermissionsOnDestination(@PathParam("prop } } - protected void validateAdminAndClientPermission(DestinationName destination) { + protected void validateAdminAndClientPermission() { try { - validateAdminAccessOnProperty(destination.getProperty()); + validateAdminAccessOnProperty(destinationName.getProperty()); } catch (Exception ve) { try { - checkAuthorization(pulsar(), destination, clientAppId(), clientAuthData()); + checkAuthorization(pulsar(), destinationName, clientAppId(), clientAuthData()); } catch (RestException re) { throw re; } catch (Exception e) { // unknown error marked as internal server error - log.warn("Unexpected error while authorizing request. destination={}, role={}. Error: {}", destination, - clientAppId(), e.getMessage(), e); + log.warn("Unexpected error while authorizing request. destination={}, role={}. Error: {}", + destinationName, clientAppId(), e.getMessage(), e); throw new RestException(e); } } } - protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) { - validateAdminAccessOnProperty(fqdn.getProperty()); - validateDestinationOwnership(fqdn, authoritative); + public void validateAdminOperationOnDestination(boolean authoritative) { + validateAdminAccessOnProperty(destinationName.getProperty()); + validateDestinationOwnership(destinationName, authoritative); } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}") - @ApiOperation(value = "Grant a new permission to a role on a single destination.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace doesn't exist"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void grantPermissionsOnDestination(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, @PathParam("role") String role, Set actions) { - destination = decode(destination); + protected void internalGrantPermissionsOnDestination(String role, Set actions) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges - validateAdminAccessOnProperty(property); + validateAdminAccessOnProperty(namespaceName.getProperty()); validatePoliciesReadOnlyAccess(); - String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString(); + String destinationUri = destinationName.toString(); try { Stat nodeStat = new Stat(); - byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat); + byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat); Policies policies = jsonMapper().readValue(content, Policies.class); if (!policies.auth_policies.destination_auth.containsKey(destinationUri)) { @@ -314,11 +263,11 @@ public void grantPermissionsOnDestination(@PathParam("property") String property policies.auth_policies.destination_auth.get(destinationUri).put(role, actions); // Write the new policies to zookeeper - globalZk().setData(path(POLICIES, property, cluster, namespace), jsonMapper().writeValueAsBytes(policies), + globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); // invalidate the local cache to force update - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); log.info("[{}] Successfully granted access for role {}: {} - destination {}", clientAppId(), role, actions, destinationUri); @@ -333,27 +282,17 @@ public void grantPermissionsOnDestination(@PathParam("property") String property } } - @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}") - @ApiOperation(value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination" - + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace doesn't exist"), - @ApiResponse(code = 412, message = "Permissions are not set at the destination level") }) - public void revokePermissionsOnDestination(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, @PathParam("role") String role) { - destination = decode(destination); + protected void internalRevokePermissionsOnDestination(String role) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges - validateAdminAccessOnProperty(property); + validateAdminAccessOnProperty(namespaceName.getProperty()); validatePoliciesReadOnlyAccess(); - String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString(); + String destinationUri = destinationName.toString(); Stat nodeStat = new Stat(); Policies policies; try { - byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat); + byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat); policies = jsonMapper().readValue(content, Policies.class); } catch (KeeperException.NoNodeException e) { log.warn("[{}] Failed to revoke permissions on destination {}: Namespace does not exist", clientAppId(), @@ -375,7 +314,7 @@ public void revokePermissionsOnDestination(@PathParam("property") String propert try { // Write the new policies to zookeeper - String namespacePath = path(POLICIES, property, cluster, namespace); + String namespacePath = path(POLICIES, namespaceName.toString()); globalZk().setData(namespacePath, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); // invalidate the local cache to force update @@ -390,33 +329,24 @@ public void revokePermissionsOnDestination(@PathParam("property") String propert } } - @PUT - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") - @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 409, message = "Partitioned topic already exist") }) - public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, int numPartitions, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.getProperty()); + protected void internalCreatePartitionedTopic(int numPartitions, boolean authoritative) { + validateAdminAccessOnProperty(destinationName.getProperty()); if (numPartitions <= 1) { throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1"); } try { - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(), - dn.getEncodedLocalName()); + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), + destinationName.getEncodedLocalName()); byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); zkCreateOptimistic(path, data); // we wait for the data to be synced in all quorums and the observers Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS); - log.info("[{}] Successfully created partitioned topic {}", clientAppId(), dn); + log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName); } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), dn); + log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName); throw new RestException(Status.CONFLICT, "Partitioned topic already exist"); } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), dn, e); + log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e); throw new RestException(e); } } @@ -430,77 +360,47 @@ public void createPartitionedTopic(@PathParam("property") String property, @Path * recreate them at application so, newly created producers and consumers can connect to newly added partitions as * well. Therefore, it can violate partition ordering at producers until all producers are restarted at application. * - * @param property - * @param cluster - * @param namespace - * @param destination * @param numPartitions */ - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") - @ApiOperation(value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 409, message = "Partitioned topic does not exist") }) - public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - int numPartitions) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.getProperty()); - if (dn.isGlobal()) { - log.error("[{}] Update partitioned-topic is forbidden on global namespace {}", clientAppId(), dn); + protected void internalUpdatePartitionedTopic(int numPartitions) { + validateAdminAccessOnProperty(destinationName.getProperty()); + if (destinationName.isGlobal()) { + log.error("[{}] Update partitioned-topic is forbidden on global namespace {}", clientAppId(), + destinationName); throw new RestException(Status.FORBIDDEN, "Update forbidden on global namespace"); } if (numPartitions <= 1) { throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1"); } try { - updatePartitionedTopic(dn, numPartitions).get(); + updatePartitionedTopic(destinationName, numPartitions).get(); } catch (Exception e) { if (e.getCause() instanceof RestException) { throw (RestException) e.getCause(); } - log.error("[{}] Failed to update partitioned topic {}", clientAppId(), dn, e.getCause()); + log.error("[{}] Failed to update partitioned topic {}", clientAppId(), destinationName, e.getCause()); throw new RestException(e.getCause()); } } - @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") - @ApiOperation(value = "Get partitioned topic metadata.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) - public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(property, cluster, namespace, destination, authoritative); + protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean authoritative) { + PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(destinationName, authoritative); if (metadata.partitions > 1) { validateClientVersion(); } return metadata; } - @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") - @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Partitioned topic does not exist") }) - public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.getProperty()); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected void internalDeletePartitionedTopic(boolean authoritative) { + validateAdminAccessOnProperty(destinationName.getProperty()); + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); int numPartitions = partitionMetadata.partitions; if (numPartitions > 0) { final CompletableFuture future = new CompletableFuture<>(); final AtomicInteger count = new AtomicInteger(numPartitions); try { for (int i = 0; i < numPartitions; i++) { - DestinationName dn_partition = dn.getPartition(i); + DestinationName dn_partition = destinationName.getPartition(i); pulsar().getAdminClient().persistentTopics().deleteAsync(dn_partition.toString()) .whenComplete((r, ex) -> { if (ex != null) { @@ -538,63 +438,45 @@ public void deletePartitionedTopic(@PathParam("property") String property, @Path } // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(), - dn.getEncodedLocalName()); + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), + destinationName.getEncodedLocalName()); try { globalZk().delete(path, -1); globalZkCache().invalidate(path); // we wait for the data to be synced in all quorums and the observers Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS); - log.info("[{}] Deleted partitioned topic {}", clientAppId(), dn); + log.info("[{}] Deleted partitioned topic {}", clientAppId(), destinationName); } catch (KeeperException.NoNodeException nne) { throw new RestException(Status.NOT_FOUND, "Partitioned topic does not exist"); } catch (Exception e) { - log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), dn, e); + log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), destinationName, e); throw new RestException(e); } } - @PUT - @Path("/{property}/{cluster}/{namespace}/{destination}/unload") - @ApiOperation(value = "Unload a topic") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - log.info("[{}] Unloading topic {}/{}/{}/{}", clientAppId(), property, cluster, namespace, destination); - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - unloadTopic(dn, authoritative); + protected void internalUnloadTopic(boolean authoritative) { + log.info("[{}] Unloading topic {}", clientAppId(), destinationName); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + unloadTopic(destinationName, authoritative); } - @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}") - @ApiOperation(value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist"), - @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") }) - public void deleteTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); - if (dn.isGlobal()) { + protected void internalDeleteTopic(boolean authoritative) { + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); + if (destinationName.isGlobal()) { // Delete is disallowed on global topic - log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), dn); + log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), destinationName); throw new RestException(Status.FORBIDDEN, "Delete forbidden on global namespace"); } + try { topic.delete().get(); - log.info("[{}] Successfully removed topic {}", clientAppId(), dn); + log.info("[{}] Successfully removed topic {}", clientAppId(), destinationName); } catch (Exception e) { Throwable t = e.getCause(); - log.error("[{}] Failed to get delete topic {}", clientAppId(), dn, t); + log.error("[{}] Failed to get delete topic {}", clientAppId(), destinationName, t); if (t instanceof TopicBusyException) { throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions"); } else { @@ -603,39 +485,30 @@ public void deleteTopic(@PathParam("property") String property, @PathParam("clus } } - @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions") - @ApiOperation(value = "Get the list of persistent subscriptions for a given topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public List getSubscriptions(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); + protected List internalGetSubscriptions(boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); } + List subscriptions = Lists.newArrayList(); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { try { // get the subscriptions only from the 1st partition since all the other partitions will have the same // subscriptions - subscriptions.addAll( - pulsar().getAdminClient().persistentTopics().getSubscriptions(dn.getPartition(0).toString())); + subscriptions.addAll(pulsar().getAdminClient().persistentTopics() + .getSubscriptions(destinationName.getPartition(0).toString())); } catch (Exception e) { throw new RestException(e); } } else { - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); try { topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName)); } catch (Exception e) { - log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), dn); + log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), destinationName); throw new RestException(e); } } @@ -643,60 +516,32 @@ public List getSubscriptions(@PathParam("property") String property, @Pa return subscriptions; } - @GET - @Path("{property}/{cluster}/{namespace}/{destination}/stats") - @ApiOperation(value = "Get the stats for the topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public PersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAndClientPermission(dn); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - validateDestinationOwnership(dn, authoritative); - Topic topic = getTopicReference(dn); + protected PersistentTopicStats internalGetStats(boolean authoritative) { + validateAdminAndClientPermission(); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + validateDestinationOwnership(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); return topic.getStats(); } - @GET - @Path("{property}/{cluster}/{namespace}/{destination}/internalStats") - @ApiOperation(value = "Get the internal stats for the topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAndClientPermission(dn); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - validateDestinationOwnership(dn, authoritative); - Topic topic = getTopicReference(dn); + protected PersistentTopicInternalStats internalGetInternalStats(boolean authoritative) { + validateAdminAndClientPermission(); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + validateDestinationOwnership(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); return topic.getInternalStats(); } - @GET - @Path("{property}/{cluster}/{namespace}/{destination}/internal-info") - @ApiOperation(value = "Get the internal stats for the topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @Suspended AsyncResponse asyncResponse) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.getProperty()); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - String managedLedger = dn.getPersistenceNamingEncoding(); + protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse) { + validateAdminAccessOnProperty(destinationName.getProperty()); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + String managedLedger = destinationName.getPersistenceNamingEncoding(); pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() { @Override public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { @@ -712,32 +557,21 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) { }, null); } - @GET - @Path("{property}/{cluster}/{namespace}/{destination}/partitioned-stats") - @ApiOperation(value = "Get the stats for the partitioned topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected PartitionedTopicStats internalGetPartitionedStats(boolean authoritative) { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions == 0) { throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found"); } - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); } PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata); try { for (int i = 0; i < partitionMetadata.partitions; i++) { PersistentTopicStats partitionStats = pulsar().getAdminClient().persistentTopics() - .getStats(dn.getPartition(i).toString()); + .getStats(destinationName.getPartition(i).toString()); stats.add(partitionStats); - stats.partitions.put(dn.getPartition(i).toString(), partitionStats); + stats.partitions.put(destinationName.getPartition(i).toString(), partitionStats); } } catch (Exception e) { throw new RestException(e); @@ -745,28 +579,16 @@ public PartitionedTopicStats getPartitionedStats(@PathParam("property") String p return stats; } - @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}") - @ApiOperation(value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic does not exist"), - @ApiResponse(code = 412, message = "Subscription has active consumers") }) - public void deleteSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected void internalDeleteSubscription(String subName, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { try { for (int i = 0; i < partitionMetadata.partitions; i++) { - pulsar().getAdminClient().persistentTopics().deleteSubscription(dn.getPartition(i).toString(), - subName); + pulsar().getAdminClient().persistentTopics() + .deleteSubscription(destinationName.getPartition(i).toString(), subName); } } catch (Exception e) { if (e instanceof NotFoundException) { @@ -774,18 +596,18 @@ public void deleteSubscription(@PathParam("property") String property, @PathPara } else if (e instanceof PreconditionFailedException) { throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers"); } else { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), dn, subName, e); + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e); throw new RestException(e); } } } else { - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); try { Subscription sub = topic.getSubscription(subName); checkNotNull(sub); sub.delete().get(); - log.info("[{}][{}] Deleted subscription {}", clientAppId(), dn, subName); + log.info("[{}][{}] Deleted subscription {}", clientAppId(), destinationName, subName); } catch (Exception e) { Throwable t = e.getCause(); if (e instanceof NullPointerException) { @@ -793,43 +615,30 @@ public void deleteSubscription(@PathParam("property") String property, @PathPara } else if (t instanceof SubscriptionBusyException) { throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers"); } else { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), dn, subName, e); + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e); throw new RestException(t); } } } - } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all") - @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), - @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) - public void skipAllMessages(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected void internalSkipAllMessages(String subName, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { try { for (int i = 0; i < partitionMetadata.partitions; i++) { - pulsar().getAdminClient().persistentTopics().skipAllMessages(dn.getPartition(i).toString(), - subName); + pulsar().getAdminClient().persistentTopics() + .skipAllMessages(destinationName.getPartition(i).toString(), subName); } } catch (Exception e) { throw new RestException(e); } } else { - validateAdminOperationOnDestination(dn, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); try { if (subName.startsWith(topic.replicatorPrefix)) { String remoteCluster = PersistentReplicator.getRemoteCluster(subName); @@ -841,38 +650,26 @@ public void skipAllMessages(@PathParam("property") String property, @PathParam(" checkNotNull(sub); sub.clearBacklog().get(); } - log.info("[{}] Cleared backlog on {} {}", clientAppId(), dn, subName); + log.info("[{}] Cleared backlog on {} {}", clientAppId(), destinationName, subName); } catch (NullPointerException npe) { throw new RestException(Status.NOT_FOUND, "Subscription not found"); } catch (Exception exception) { - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), dn, subName, exception); + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), destinationName, subName, exception); throw new RestException(exception); } } - } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}") - @ApiOperation(value = "Skip messages on a topic subscription.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) - public void skipMessages(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("numMessages") int numMessages, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected void internalSkipMessages(String subName, int numMessages, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { throw new RestException(Status.METHOD_NOT_ALLOWED, "Skip messages on a partitioned topic is not allowed"); } - validateAdminOperationOnDestination(dn, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); try { if (subName.startsWith(topic.replicatorPrefix)) { String remoteCluster = PersistentReplicator.getRemoteCluster(subName); @@ -884,85 +681,51 @@ public void skipMessages(@PathParam("property") String property, @PathParam("clu checkNotNull(sub); sub.skipMessages(numMessages).get(); } - log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, dn, subName); + log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, destinationName, subName); } catch (NullPointerException npe) { throw new RestException(Status.NOT_FOUND, "Subscription not found"); } catch (Exception exception) { - log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, dn, subName, exception); + log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, destinationName, subName, + exception); throw new RestException(exception); } } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") - @ApiOperation(value = "Expire messages on a topic subscription.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) - public void expireTopicMessages(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative); - } - - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}") - @ApiOperation(value = "Expire messages on all subscriptions of topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) - public void expireMessagesForAllSubscriptions(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destinationName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - final String destination = decode(destinationName); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected void internalExpireMessagesForAllSubscriptions(int expireTimeInSeconds, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { try { // expire messages for each partition destination for (int i = 0; i < partitionMetadata.partitions; i++) { - pulsar().getAdminClient().persistentTopics() - .expireMessagesForAllSubscriptions(dn.getPartition(i).toString(), expireTimeInSeconds); + pulsar().getAdminClient().persistentTopics().expireMessagesForAllSubscriptions( + destinationName.getPartition(i).toString(), expireTimeInSeconds); } } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds, dn, - e); + log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds, + destinationName, e); throw new RestException(e); } } else { // validate ownership and redirect if current broker is not owner - validateAdminOperationOnDestination(dn, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); topic.getReplicators().forEach((subName, replicator) -> { - expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative); + internalExpireMessages(subName, expireTimeInSeconds, authoritative); }); topic.getSubscriptions().forEach((subName, subscriber) -> { - expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative); + internalExpireMessages(subName, expireTimeInSeconds, authoritative); }); } } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}") - @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic/Subscription does not exist") }) - public void resetCursor(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("timestamp") long timestamp, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected void internalResetCursor(String subName, long timestamp, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { int numParts = partitionMetadata.partitions; @@ -970,8 +733,8 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus Exception partitionException = null; try { for (int i = 0; i < numParts; i++) { - pulsar().getAdminClient().persistentTopics().resetCursor(dn.getPartition(i).toString(), subName, - timestamp); + pulsar().getAdminClient().persistentTopics().resetCursor(destinationName.getPartition(i).toString(), + subName, timestamp); } } catch (PreconditionFailedException pfe) { // throw the last exception if all partitions get this error @@ -979,25 +742,25 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus ++numPartException; partitionException = pfe; } catch (Exception e) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName, - timestamp, e); + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), + destinationName, subName, timestamp, e); throw new RestException(e); } // report an error to user if unable to reset for all partitions if (numPartException == numParts) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName, - timestamp, partitionException); + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), + destinationName, subName, timestamp, partitionException); throw new RestException(Status.PRECONDITION_FAILED, partitionException.getMessage()); } else if (numPartException > 0) { log.warn("[{}][{}] partial errors for reset cursor on subscription {} to time {} - ", clientAppId(), - destination, subName, timestamp, partitionException); + destinationName, subName, timestamp, partitionException); } } else { - validateAdminOperationOnDestination(dn, authoritative); - log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destination, + validateAdminOperationOnDestination(authoritative); + log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destinationName, subName, timestamp); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); if (topic == null) { throw new RestException(Status.NOT_FOUND, "Topic not found"); } @@ -1005,11 +768,12 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus PersistentSubscription sub = topic.getSubscription(subName); checkNotNull(sub); sub.resetCursor(timestamp).get(); - log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), dn, subName, timestamp); + log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), destinationName, subName, + timestamp); } catch (Exception e) { Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName, - timestamp, e); + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), + destinationName, subName, timestamp, e); if (e instanceof NullPointerException) { throw new RestException(Status.NOT_FOUND, "Subscription not found"); } else if (e instanceof NotAllowedException) { @@ -1024,26 +788,14 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus } } - @PUT - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subscriptionName}") - @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "Creates a subscription on the topic at the specified message id") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic/Subscription does not exist"), - @ApiResponse(code = 405, message = "Not supported for partitioned topics") }) - public void createSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subscriptionName") String subscriptionName, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) throws PulsarServerException { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), destination, + protected void internalCreateSubscription(String subscriptionName, MessageIdImpl messageId, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), destinationName, subscriptionName, messageId); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); try { if (partitionMetadata.partitions > 0) { @@ -1052,15 +804,16 @@ public void createSubscription(@PathParam("property") String property, @PathPara PulsarAdmin admin = pulsar().getAdminClient(); for (int i = 0; i < partitionMetadata.partitions; i++) { - futures.add(admin.persistentTopics().createSubscriptionAsync(dn.getPartition(i).toString(), + futures.add(admin.persistentTopics().createSubscriptionAsync( + destinationName.getPartition(i).toString(), subscriptionName, messageId)); } FutureUtil.waitForAll(futures).join(); } else { - validateAdminOperationOnDestination(dn, authoritative); + validateAdminOperationOnDestination(authoritative); - PersistentTopic topic = (PersistentTopic) getOrCreateTopic(dn); + PersistentTopic topic = (PersistentTopic) getOrCreateTopic(destinationName); if (topic.getSubscriptions().containsKey(subscriptionName)) { throw new RestException(Status.CONFLICT, "Subscription already exists for topic"); @@ -1069,13 +822,13 @@ public void createSubscription(@PathParam("property") String property, @PathPara PersistentSubscription subscription = (PersistentSubscription) topic .createSubscription(subscriptionName).get(); subscription.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get(); - log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(), dn, - subscriptionName, messageId); + log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(), + destinationName, subscriptionName, messageId); } } catch (Exception e) { Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), dn, subscriptionName, - messageId, e); + log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), + destinationName, subscriptionName, messageId, e); if (t instanceof SubscriptionInvalidCursorPosition) { throw new RestException(Status.PRECONDITION_FAILED, "Unable to find position for position specified: " + t.getMessage()); @@ -1085,34 +838,23 @@ public void createSubscription(@PathParam("property") String property, @PathPara } } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor") - @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic/Subscription does not exist"), - @ApiResponse(code = 405, message = "Not supported for partitioned topics") }) - public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destination, + protected void internalResetCursorOnPosition(String subName, boolean authoritative, MessageIdImpl messageId) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destinationName, subName, messageId); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { - log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), dn, subName); + log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), destinationName, + subName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Reset-cursor at position is not allowed for partitioned-topic"); } else { - validateAdminOperationOnDestination(dn, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); if (topic == null) { throw new RestException(Status.NOT_FOUND, "Topic not found"); } @@ -1120,12 +862,12 @@ public void resetCursorOnPosition(@PathParam("property") String property, @PathP PersistentSubscription sub = topic.getSubscription(subName); checkNotNull(sub); sub.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get(); - log.info("[{}][{}] successfully reset cursor on subscription {} to position {}", clientAppId(), dn, - subName, messageId); + log.info("[{}][{}] successfully reset cursor on subscription {} to position {}", clientAppId(), + destinationName, subName, messageId); } catch (Exception e) { Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), dn, - subName, messageId, e); + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), + destinationName, subName, messageId, e); if (e instanceof NullPointerException) { throw new RestException(Status.NOT_FOUND, "Subscription not found"); } else if (t instanceof SubscriptionInvalidCursorPosition) { @@ -1138,32 +880,22 @@ public void resetCursorOnPosition(@PathParam("property") String property, @PathP } } - @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}") - @ApiOperation(value = "Peek nth message on a topic subscription.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") }) - public Response peekNthMessage(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected Response internalPeekNthMessage(String subName, int messagePosition, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { throw new RestException(Status.METHOD_NOT_ALLOWED, "Peek messages on a partitioned topic is not allowed"); } - validateAdminOperationOnDestination(dn, authoritative); - if (!(getTopicReference(dn) instanceof PersistentTopic)) { - log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), dn, subName); + validateAdminOperationOnDestination(authoritative); + if (!(getTopicReference(destinationName) instanceof PersistentTopic)) { + log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName, + subName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Skip messages on a non-persistent topic is not allowed"); } - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); PersistentReplicator repl = null; PersistentSubscription sub = null; Entry entry = null; @@ -1223,8 +955,8 @@ public void write(OutputStream output) throws IOException, WebApplicationExcepti } catch (NullPointerException npe) { throw new RestException(Status.NOT_FOUND, "Message not found"); } catch (Exception exception) { - log.error("[{}] Failed to get message at position {} from {} {}", clientAppId(), messagePosition, dn, - subName, exception); + log.error("[{}] Failed to get message at position {} from {} {}", clientAppId(), messagePosition, + destinationName, subName, exception); throw new RestException(exception); } finally { if (entry != null) { @@ -1233,37 +965,26 @@ public void write(OutputStream output) throws IOException, WebApplicationExcepti } } - @GET - @Path("{property}/{cluster}/{namespace}/{destination}/backlog") - @ApiOperation(value = "Get estimated backlog for offline topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - validateAdminAccessOnProperty(property); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); + protected PersistentOfflineTopicStats internalGetBacklog(boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); } // Validate that namespace exists, throw 404 if it doesn't exist // note that we do not want to load the topic and hence skip validateAdminOperationOnDestination() try { - policiesCache().get(path(POLICIES, property, cluster, namespace)); + policiesCache().get(path(POLICIES, namespaceName.toString())); } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get topic backlog {}/{}/{}: Namespace does not exist", clientAppId(), property, - cluster, namespace); + log.warn("[{}] Failed to get topic backlog {}: Namespace does not exist", clientAppId(), namespaceName); throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); } catch (Exception e) { - log.error("[{}] Failed to get topic backlog {}/{}/{}", clientAppId(), property, cluster, namespace, e); + log.error("[{}] Failed to get topic backlog {}", clientAppId(), namespaceName, e); throw new RestException(e); } - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); + PersistentOfflineTopicStats offlineTopicStats = null; try { - offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(dn); + offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(destinationName); if (offlineTopicStats != null) { // offline topic stat has a cost - so use cached value until TTL long elapsedMs = System.currentTimeMillis() - offlineTopicStats.statGeneratedAt.getTime(); @@ -1271,75 +992,62 @@ public PersistentOfflineTopicStats getBacklog(@PathParam("property") String prop return offlineTopicStats; } } - final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(dn).get(); + final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(destinationName) + .get(); ManagedLedgerOfflineBacklog offlineTopicBacklog = new ManagedLedgerOfflineBacklog(config.getDigestType(), config.getPassword(), pulsar().getAdvertisedAddress(), false); - offlineTopicStats = offlineTopicBacklog - .estimateUnloadedTopicBacklog((ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), dn); - pulsar().getBrokerService().cacheOfflineTopicStats(dn, offlineTopicStats); + offlineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog( + (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), destinationName); + pulsar().getBrokerService().cacheOfflineTopicStats(destinationName, offlineTopicStats); } catch (Exception exception) { throw new RestException(exception); } return offlineTopicStats; } - @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/terminate") - @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more " - + "messages to be published and will let consumer to drain existing messages in backlog") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), - @ApiResponse(code = 404, message = "Topic does not exist") }) - public MessageId terminate(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + protected MessageId internalTerminate(boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { throw new RestException(Status.METHOD_NOT_ALLOWED, "Termination of a partitioned topic is not allowed"); } - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); try { return ((PersistentTopic) topic).terminate().get(); } catch (Exception exception) { - log.error("[{}] Failed to terminated topic {}", clientAppId(), dn, exception); + log.error("[{}] Failed to terminated topic {}", clientAppId(), destinationName, exception); throw new RestException(exception); } } - public void expireMessages(String property, String cluster, String namespace, String destination, String subName, - int expireTimeInSeconds, boolean authoritative) { - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); + protected void internalExpireMessages(String subName, int expireTimeInSeconds, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative); if (partitionMetadata.partitions > 0) { // expire messages for each partition destination try { for (int i = 0; i < partitionMetadata.partitions; i++) { - pulsar().getAdminClient().persistentTopics().expireMessages(dn.getPartition(i).toString(), subName, - expireTimeInSeconds); + pulsar().getAdminClient().persistentTopics() + .expireMessages(destinationName.getPartition(i).toString(), subName, expireTimeInSeconds); } } catch (Exception e) { throw new RestException(e); } } else { // validate ownership and redirect if current broker is not owner - validateAdminOperationOnDestination(dn, authoritative); - if (!(getTopicReference(dn) instanceof PersistentTopic)) { - log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), dn, subName); + validateAdminOperationOnDestination(authoritative); + if (!(getTopicReference(destinationName) instanceof PersistentTopic)) { + log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName, + subName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Expire messages on a non-persistent topic is not allowed"); } - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); try { if (subName.startsWith(topic.replicatorPrefix)) { String remoteCluster = PersistentReplicator.getRemoteCluster(subName); @@ -1351,13 +1059,13 @@ public void expireMessages(String property, String cluster, String namespace, St checkNotNull(sub); sub.expireMessages(expireTimeInSeconds); } - log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds, dn, - subName); + log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds, + destinationName, subName); } catch (NullPointerException npe) { throw new RestException(Status.NOT_FOUND, "Subscription not found"); } catch (Exception exception) { log.error("[{}] Failed to expire messages up to {} on {} with subscription {} {}", clientAppId(), - expireTimeInSeconds, dn, subName, exception); + expireTimeInSeconds, destinationName, subName, exception); throw new RestException(exception); } } @@ -1386,8 +1094,8 @@ public static CompletableFuture getPartitionedTopicMet throw ex; } - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), - dn.getNamespacePortion(), "persistent", dn.getEncodedLocalName()); + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getNamespace(), + "persistent", dn.getEncodedLocalName()); // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can // serve/redirect request else fail partitioned-metadata-request so, client fails while creating @@ -1409,7 +1117,7 @@ public static CompletableFuture getPartitionedTopicMet return metadataFuture; } - /** + /** * Get the Topic object reference from the Pulsar broker */ private Topic getTopicReference(DestinationName dn) { @@ -1485,8 +1193,10 @@ private CompletableFuture updatePartitionedTopic(DestinationName dn, int n /** * It creates subscriptions for new partitions of existing partitioned-topics * - * @param dn : topic-name: persistent://prop/cluster/ns/topic - * @param numPartitions : number partitions for the topics + * @param dn + * : topic-name: persistent://prop/cluster/ns/topic + * @param numPartitions + * : number partitions for the topics */ private CompletableFuture createSubscriptions(DestinationName dn, int numPartitions) { String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Properties.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java similarity index 95% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Properties.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java index 359f3002c7de5..6845e91b45f29 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Properties.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java @@ -16,22 +16,19 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; +package org.apache.pulsar.broker.admin.impl; -import java.util.Collections; import java.util.List; -import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.PUT; import javax.ws.rs.Path; import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response.Status; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.NamedEntity; import org.apache.pulsar.common.policies.data.PropertyAdmin; @@ -41,17 +38,12 @@ import org.slf4j.LoggerFactory; import com.google.common.collect.Lists; -import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -@Path("/properties") -@Produces(MediaType.APPLICATION_JSON) -@Consumes(MediaType.APPLICATION_JSON) -@Api(value = "/properties", description = "Properties admin apis", tags = "properties") -public class Properties extends AdminResource { +public class PropertiesBase extends AdminResource { @GET @ApiOperation(value = "Get the list of properties.", response = String.class, responseContainer = "List") @@ -207,5 +199,5 @@ public void deleteProperty(@PathParam("property") String property) { } } - private static final Logger log = LoggerFactory.getLogger(Properties.class); + private static final Logger log = LoggerFactory.getLogger(PropertiesBase.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java similarity index 53% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java index 3eed02328def6..9c5a54f8ce592 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java @@ -16,43 +16,20 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; - -import javax.ws.rs.Consumes; -import javax.ws.rs.DELETE; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; +package org.apache.pulsar.broker.admin.impl; + import javax.ws.rs.core.Response.Status; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.NamespaceBundle; -import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.ResourceQuota; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiOperation; -import io.swagger.annotations.ApiResponse; -import io.swagger.annotations.ApiResponses; - -@Path("/resource-quotas") -@Produces(MediaType.APPLICATION_JSON) -@Consumes(MediaType.APPLICATION_JSON) -@Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas") -public class ResourceQuotas extends AdminResource { +public abstract class ResourceQuotasBase extends NamespacesBase { - private static final Logger log = LoggerFactory.getLogger(ResourceQuotas.class); - - @GET - @ApiOperation(value = "Get the default quota", response = String.class, responseContainer = "Set") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public ResourceQuota getDefaultResourceQuota() throws Exception { validateSuperUserAccess(); try { @@ -64,9 +41,6 @@ public ResourceQuota getDefaultResourceQuota() throws Exception { } - @POST - @ApiOperation(value = "Set the default quota", response = String.class, responseContainer = "Set") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setDefaultResourceQuota(ResourceQuota quota) throws Exception { validateSuperUserAccess(); validatePoliciesReadOnlyAccess(); @@ -78,25 +52,18 @@ public void setDefaultResourceQuota(ResourceQuota quota) throws Exception { } } - @GET - @Path("/{property}/{cluster}/{namespace}/{bundle}") - @ApiOperation(value = "Get resource quota of a namespace bundle.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace does not exist") }) - public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange) { + @SuppressWarnings("deprecation") + protected ResourceQuota internalGetNamespaceBundleResourceQuota(String bundleRange) { validateSuperUserAccess(); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); } - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange); + NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange); try { return pulsar().getLocalZkCacheService().getResourceQuotaCache().getQuota(nsBundle); @@ -106,26 +73,19 @@ public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") Stri } } - @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}") - @ApiOperation(value = "Set resource quota on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void setNamespaceBundleResourceQuota(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange, ResourceQuota quota) { + @SuppressWarnings("deprecation") + protected void internalSetNamespaceBundleResourceQuota(String bundleRange, ResourceQuota quota) { validateSuperUserAccess(); validatePoliciesReadOnlyAccess(); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); } - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange); + NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange); try { pulsar().getLocalZkCacheService().getResourceQuotaCache().setQuota(nsBundle, quota); @@ -142,26 +102,19 @@ public void setNamespaceBundleResourceQuota(@PathParam("property") String proper } - @DELETE - @Path("/{property}/{cluster}/{namespace}/{bundle}") - @ApiOperation(value = "Remove resource quota for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 409, message = "Concurrent modification") }) - public void removeNamespaceBundleResourceQuota(@PathParam("property") String property, - @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange) { + @SuppressWarnings("deprecation") + protected void internalRemoveNamespaceBundleResourceQuota(String bundleRange) { validateSuperUserAccess(); validatePoliciesReadOnlyAccess(); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); } - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange); + NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange); try { pulsar().getLocalZkCacheService().getResourceQuotaCache().unsetQuota(nsBundle); @@ -177,4 +130,6 @@ public void removeNamespaceBundleResourceQuota(@PathParam("property") String pro throw new RestException(e); } } + + private static final Logger log = LoggerFactory.getLogger(ResourceQuotasBase.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/BrokerStats.java new file mode 100644 index 0000000000000..97ebbd9a7f29d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/BrokerStats.java @@ -0,0 +1,32 @@ +/** + * 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.v1; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.BrokerStatsBase; + +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/broker-stats") +@Api(value = "/broker-stats", description = "Stats for broker", tags = "broker-stats") +@Produces(MediaType.APPLICATION_JSON) +public class BrokerStats extends BrokerStatsBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Brokers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Brokers.java new file mode 100644 index 0000000000000..c5d712cb32d0f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Brokers.java @@ -0,0 +1,29 @@ +/** + * 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.v1; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.BrokersBase; + +import javax.ws.rs.Path; + +@Path("/brokers") +@Api(value = "/brokers", description = "BrokersBase admin apis", tags = "brokers") +public class Brokers extends BrokersBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Clusters.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Clusters.java new file mode 100644 index 0000000000000..7cd5ccd454017 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Clusters.java @@ -0,0 +1,32 @@ +/** + * 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.v1; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.ClustersBase; + +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/clusters") +@Api(value = "/clusters", description = "Cluster admin apis", tags = "clusters") +@Produces(MediaType.APPLICATION_JSON) +public class Clusters extends ClustersBase { +} 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 new file mode 100644 index 0000000000000..3ae1d4bca7495 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java @@ -0,0 +1,711 @@ +/** + * 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.v1; + +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.POLICIES_ROOT; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response.Status; + +import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.admin.impl.NamespacesBase; +import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; +import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.PersistencePolicies; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +@Path("/namespaces") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces") +public class Namespaces extends NamespacesBase { + + @GET + @Path("/{property}") + @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property doesn't exist") }) + public List getPropertyNamespaces(@PathParam("property") String property) { + return internalGetPropertyNamespaces(property); + } + + @GET + @Path("/{property}/{cluster}") + @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, + @PathParam("cluster") String cluster) { + validateAdminAccessOnProperty(property); + List namespaces = Lists.newArrayList(); + if (!clusters().contains(cluster)) { + log.warn("[{}] Failed to get namespace list for property: {}/{} - Cluster does not exist", clientAppId(), + property, cluster); + throw new RestException(Status.NOT_FOUND, "Cluster does not exist"); + } + + try { + for (String namespace : globalZk().getChildren(path(POLICIES, property, cluster), false)) { + namespaces.add(String.format("%s/%s/%s", property, cluster, namespace)); + } + } catch (KeeperException.NoNodeException e) { + // NoNode means there are no namespaces for this property on the specified cluster, returning empty list + } catch (Exception e) { + log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e); + throw new RestException(e); + } + + namespaces.sort(null); + return namespaces; + } + + @GET + @Path("/{property}/{cluster}/{namespace}/destinations") + @ApiOperation(hidden = true, value = "Get the list of all the destinations under a certain namespace.", response = String.class, responseContainer = "Set") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public List getDestinations(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, cluster, namespace); + + // Validate that namespace exists, throws 404 if it doesn't exist + getNamespacePolicies(namespaceName); + + try { + return pulsar().getNamespaceService().getListOfDestinations(namespaceName); + } catch (Exception e) { + log.error("Failed to get topics list for namespace {}/{}/{}", property, cluster, namespace, e); + throw new RestException(e); + } + } + + @GET + @Path("/{property}/{cluster}/{namespace}") + @ApiOperation(hidden = true, value = "Get the dump all the policies specified for a namespace.", response = Policies.class) + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public Policies getPolicies(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, cluster, namespace); + return getNamespacePolicies(namespaceName); + } + + @SuppressWarnings("deprecation") + @PUT + @Path("/{property}/{cluster}/{namespace}") + @ApiOperation(hidden = true, value = "Creates a new empty namespace with no policies attached.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace already exists"), + @ApiResponse(code = 412, message = "Namespace name is not valid") }) + public void createNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, BundlesData initialBundles) { + validateNamespaceName(property, cluster, namespace); + + if (!namespaceName.isGlobal()) { + // If the namespace is non global, make sure property has the access on the cluster. For global namespace, + // same check is made at the time of setting replication. + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + Policies policies = new Policies(); + if (initialBundles != null && initialBundles.getNumBundles() > 0) { + if (initialBundles.getBoundaries() == null || initialBundles.getBoundaries().size() == 0) { + policies.bundles = getBundles(initialBundles.getNumBundles()); + } else { + policies.bundles = validateBundlesData(initialBundles); + } + } else { + int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles(); + policies.bundles = getBundles(defaultNumberOfBundles); + } + + internalCreateNamespace(policies); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}") + @ApiOperation(hidden = true, value = "Delete a namespace and all the destinations under it.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace is not empty") }) + public void deleteNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalDeleteNamespace(authoritative); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/{bundle}") + @ApiOperation(hidden = true, value = "Delete a namespace bundle and all the destinations under it.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace bundle is not empty") }) + public void deleteNamespaceBundle(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalDeleteNamespaceBundle(bundleRange, authoritative); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/permissions") + @ApiOperation(hidden = true, value = "Retrieve the permissions for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace is not empty") }) + public Map> getPermissions(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, cluster, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + return policies.auth_policies.namespace_auth; + } + + @POST + @Path("/{property}/{cluster}/{namespace}/permissions/{role}") + @ApiOperation(hidden = true, value = "Grant a new permission to a role on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void grantPermissionOnNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("role") String role, Set actions) { + validateNamespaceName(property, cluster, namespace); + internalGrantPermissionOnNamespace(role, actions); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/permissions/{role}") + @ApiOperation(hidden = true, value = "Revoke all permissions to a role on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void revokePermissionsOnNamespace(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("role") String role) { + validateNamespaceName(property, cluster, namespace); + internalRevokePermissionsOnNamespace(role); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/replication") + @ApiOperation(hidden = true, value = "Get the replication clusters for a namespace.", response = String.class, responseContainer = "List") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is not global") }) + public List getNamespaceReplicationClusters(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, cluster, namespace); + + return internalGetNamespaceReplicationClusters(); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/replication") + @ApiOperation(hidden = true, value = "Set the replication clusters for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"), + @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") }) + public void setNamespaceReplicationClusters(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, List clusterIds) { + validateNamespaceName(property, cluster, namespace); + internalSetNamespaceReplicationClusters(clusterIds); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/messageTTL") + @ApiOperation(hidden = true, value = "Get the message TTL for the namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public int getNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, cluster, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + return policies.message_ttl_in_seconds; + } + + @POST + @Path("/{property}/{cluster}/{namespace}/messageTTL") + @ApiOperation(hidden = true, value = "Set message TTL in seconds for namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Invalid TTL") }) + public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, int messageTTL) { + validateNamespaceName(property, cluster, namespace); + internalSetNamespaceMessageTTL(messageTTL); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/antiAffinity") + @ApiOperation(value = "Set anti-affinity group for a namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Invalid antiAffinityGroup") }) + public void setNamespaceAntiAffinityGroup(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, String antiAffinityGroup) { + validateAdminAccessOnProperty(property); + validatePoliciesReadOnlyAccess(); + + log.info("[{}] Setting anti-affinity group {} for {}/{}/{}", clientAppId(), antiAffinityGroup, property, + cluster, namespace); + + if (isBlank(antiAffinityGroup)) { + throw new RestException(Status.PRECONDITION_FAILED, "antiAffinityGroup can't be empty"); + } + + NamespaceName nsName = NamespaceName.get(property, cluster, namespace); + Map.Entry policiesNode = null; + + try { + // Force to read the data s.t. the watch to the cache content is setup. + policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace)) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist")); + policiesNode.getKey().antiAffinityGroup = antiAffinityGroup; + + // Write back the new policies into zookeeper + globalZk().setData(path(POLICIES, property, cluster, namespace), + jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion()); + policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); + + log.info("[{}] Successfully updated the antiAffinityGroup {} on namespace {}/{}/{}", clientAppId(), + antiAffinityGroup, property, cluster, namespace); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to update the antiAffinityGroup for namespace {}/{}/{}: does not exist", clientAppId(), + property, cluster, namespace); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn( + "[{}] Failed to update the antiAffinityGroup on namespace {}/{}/{} expected policy node version={} : concurrent modification", + clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion()); + + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to update the antiAffinityGroup on namespace {}/{}/{}", clientAppId(), property, cluster, + namespace, e); + throw new RestException(e); + } + } + + @GET + @Path("/{property}/{cluster}/{namespace}/antiAffinity") + @ApiOperation(value = "Get anti-affinity group of a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public String getNamespaceAntiAffinityGroup(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + return getNamespacePolicies(property, cluster, namespace).antiAffinityGroup; + } + + @GET + @Path("{cluster}/antiAffinity/{group}") + @ApiOperation(value = "Get all namespaces that are grouped by given anti-affinity group in a given cluster. api can be only accessed by admin of any of the existing property") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 412, message = "Cluster not exist/Anti-affinity group can't be empty.") }) + public List getAntiAffinityNamespaces(@PathParam("cluster") String cluster, + @PathParam("group") String antiAffinityGroup, @QueryParam("property") String property) { + validateAdminAccessOnProperty(property); + + log.info("[{}]-{} Finding namespaces for {} in {}", clientAppId(), property, antiAffinityGroup, cluster); + + if (isBlank(antiAffinityGroup)) { + throw new RestException(Status.PRECONDITION_FAILED, "anti-affinity group can't be empty."); + } + validateClusterExists(cluster); + List namespaces = Lists.newArrayList(); + try { + for (String prop : globalZk().getChildren(POLICIES_ROOT, false)) { + for (String namespace : globalZk().getChildren(path(POLICIES, prop, cluster), false)) { + Optional policies = policiesCache() + .get(AdminResource.path(POLICIES, prop, cluster, namespace)); + if (policies.isPresent() && antiAffinityGroup.equalsIgnoreCase(policies.get().antiAffinityGroup)) { + namespaces.add(String.format("%s/%s/%s", prop, cluster, namespace)); + } + } + } + } catch (Exception e) { + log.warn("Failed to list of properties/namespace from global-zk", e); + } + return namespaces; + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/antiAffinity") + @ApiOperation(value = "Remove anti-affinity group of a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void removeNamespaceAntiAffinityGroup(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validatePoliciesReadOnlyAccess(); + + log.info("[{}] Deleting anti-affinity group for {}/{}/{}", clientAppId(), property, cluster, namespace); + + try { + Stat nodeStat = new Stat(); + final String path = path(POLICIES, property, cluster, namespace); + byte[] content = globalZk().getData(path, null, nodeStat); + Policies policies = jsonMapper().readValue(content, Policies.class); + policies.antiAffinityGroup = null; + globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); + log.info("[{}] Successfully removed anti-affinity group for a namespace={}/{}/{}", clientAppId(), property, + cluster, namespace); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: does not exist", clientAppId(), + property, cluster, namespace); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (KeeperException.BadVersionException e) { + log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: concurrent modification", + clientAppId(), property, cluster, namespace); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + log.error("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}", clientAppId(), property, + cluster, namespace, e); + throw new RestException(e); + } + } + + + @POST + @Path("/{property}/{cluster}/{namespace}/deduplication") + @ApiOperation(hidden = true, value = "Enable or disable broker side deduplication for all topics in a namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void modifyDeduplication(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, boolean enableDeduplication) { + validateNamespaceName(property, cluster, namespace); + internalModifyDeduplication(enableDeduplication); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/bundles") + @ApiOperation(hidden = true, value = "Get the bundles split data.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") }) + public BundlesData getBundlesData(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validatePoliciesReadOnlyAccess(); + validateNamespaceName(property, cluster, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + + return policies.bundles; + } + + @PUT + @Path("/{property}/{cluster}/{namespace}/unload") + @ApiOperation(hidden = true, value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker" + + "removes all producers, consumers, and connections using this namespace, and close all destinations (including" + + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the" + + "broker completes the unloading action. This operation requires strictly super user privileges, since it would" + + "result in non-persistent message loss and unexpected connection closure to the clients.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is already unloaded or Namespace has bundles activated") }) + public void unloadNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + internalUnloadNamespace(); + } + + @PUT + @Path("/{property}/{cluster}/{namespace}/{bundle}/unload") + @ApiOperation(hidden = true, value = "Unload a namespace bundle") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void unloadNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalUnloadNamespaceBundle(bundleRange, authoritative); + } + + @PUT + @Path("/{property}/{cluster}/{namespace}/{bundle}/split") + @ApiOperation(hidden = true, value = "Split a namespace bundle") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void splitNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @QueryParam("unload") @DefaultValue("false") boolean unload) { + validateNamespaceName(property, cluster, namespace); + internalSplitNamespaceBundle(bundleRange, authoritative, unload); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/dispatchRate") + @ApiOperation(hidden = true, value = "Set dispatch-rate throttling for all topics of the namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void setDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, DispatchRate dispatchRate) { + validateNamespaceName(property, cluster, namespace); + internalSetDispatchRate(dispatchRate); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/dispatchRate") + @ApiOperation(hidden = true, value = "Get dispatch-rate configured for the namespace, -1 represents not configured yet") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public DispatchRate getDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + return internalGetDispatchRate(); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/backlogQuotaMap") + @ApiOperation(hidden = true, value = "Get backlog quota map on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public Map getBacklogQuotaMap(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, cluster, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + return policies.backlog_quota_map; + } + + @POST + @Path("/{property}/{cluster}/{namespace}/backlogQuota") + @ApiOperation(hidden = true, value = " Set a backlog quota for all the destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), + @ApiResponse(code = 412, message = "Specified backlog quota exceeds retention quota. Increase retention quota and retry request") }) + public void setBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, + BacklogQuota backlogQuota) { + validateNamespaceName(property, cluster, namespace); + internalSetBacklogQuota(backlogQuotaType, backlogQuota); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/backlogQuota") + @ApiOperation(hidden = true, value = "Remove a backlog quota policy from a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void removeBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, + @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType) { + validateNamespaceName(property, cluster, namespace); + internalRemoveBacklogQuota(backlogQuotaType); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/retention") + @ApiOperation(hidden = true, value = "Get retention config on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public RetentionPolicies getRetention(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + return internalGetRetention(); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/retention") + @ApiOperation(hidden = true, value = " Set retention configuration on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), + @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) + public void setRetention(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, RetentionPolicies retention) { + validateNamespaceName(property, cluster, namespace); + internalSetRetention(retention); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/persistence") + @ApiOperation(hidden = true, value = "Set the persistence configuration for all the destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), + @ApiResponse(code = 400, message = "Invalid persistence policies") }) + public void setPersistence(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, PersistencePolicies persistence) { + validateNamespaceName(property, cluster, namespace); + internalSetPersistence(persistence); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/persistence") + @ApiOperation(hidden = true, value = "Get the persistence configuration for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public PersistencePolicies getPersistence(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + return internalGetPersistence(); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/clearBacklog") + @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalClearNamespaceBacklog(authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog") + @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBundleBacklog(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalClearNamespaceBundleBacklog(bundleRange, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}") + @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBacklogForSubscription(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("subscription") String subscription, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalClearNamespaceBacklogForSubscription(subscription, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}") + @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("subscription") String subscription, @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalClearNamespaceBundleBacklogForSubscription(subscription, bundleRange, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}") + @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalUnsubscribeNamespace(subscription, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}") + @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void unsubscribeNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalUnsubscribeNamespaceBundle(subscription, bundleRange, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode") + @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void setSubscriptionAuthMode(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, SubscriptionAuthMode subscriptionAuthMode) { + validateNamespaceName(property, cluster, namespace); + internalSetSubscriptionAuthMode(subscriptionAuthMode); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/encryptionRequired") + @ApiOperation(hidden = true, value = "Message encryption is required or not for all topics in a namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), }) + public void modifyEncryptionRequired(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, boolean encryptionRequired) { + validateNamespaceName(property, cluster, namespace); + internalModifyEncryptionRequired(encryptionRequired); + } + + private static final Logger log = LoggerFactory.getLogger(Namespaces.class); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java similarity index 79% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java index 68b4e836dcb65..c423f2184a823 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java @@ -16,15 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.admin; +package org.apache.pulsar.broker.admin.v1; import static com.google.common.base.Preconditions.checkNotNull; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import static org.apache.pulsar.common.util.Codec.decode; - -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import javax.ws.rs.DefaultValue; import javax.ws.rs.Encoded; @@ -37,12 +31,12 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response.Status; +import com.google.common.collect.Lists; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; import org.apache.pulsar.broker.web.RestException; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.naming.DestinationDomain; +import org.apache.pulsar.common.naming.Constants; import org.apache.pulsar.common.naming.DestinationName; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; @@ -55,13 +49,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; - import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + /** */ @Path("/non-persistent") @@ -69,96 +65,94 @@ @Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic") public class NonPersistentTopics extends PersistentTopics { private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class); - + @GET @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") - @ApiOperation(value = "Get partitioned topic metadata.") + @ApiOperation(hidden = true, value = "Get partitioned topic metadata.") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, + @PathParam("destination") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - return getPartitionedTopicMetadata(property, cluster, namespace, destination, authoritative); + validateDestinationName(property, cluster, namespace, encodedTopic); + return getPartitionedTopicMetadata(destinationName, authoritative); } @GET @Path("{property}/{cluster}/{namespace}/{destination}/stats") - @ApiOperation(value = "Get the stats for the topic.") + @ApiOperation(hidden = true, value = "Get the stats for the topic.") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist") }) - public NonPersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public NonPersistentTopicStats getStats(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); - return ((NonPersistentTopic)topic).getStats(); + validateDestinationName(property, cluster, namespace, encodedTopic); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); + return ((NonPersistentTopic) topic).getStats(); } @GET @Path("{property}/{cluster}/{namespace}/{destination}/internalStats") - @ApiOperation(value = "Get the internal stats for the topic.") + @ApiOperation(hidden = true, value = "Get the internal stats for the topic.") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist") }) public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, + @PathParam("destination") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); + validateDestinationName(property, cluster, namespace, encodedTopic); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); return topic.getInternalStats(); } @PUT @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") - @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") + @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Partitioned topic already exist") }) public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, int numPartitions, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.getProperty()); + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + validateAdminAccessOnProperty(destinationName.getProperty()); if (numPartitions <= 1) { throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1"); } try { - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(), - dn.getEncodedLocalName()); + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), + destinationName.getEncodedLocalName()); byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); zkCreateOptimistic(path, data); // we wait for the data to be synced in all quorums and the observers Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS); - log.info("[{}] Successfully created partitioned topic {}", clientAppId(), dn); + log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName); } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), dn); + log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName); throw new RestException(Status.CONFLICT, "Partitioned topic already exist"); } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), dn, e); + log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e); throw new RestException(e); } } @PUT @Path("/{property}/{cluster}/{namespace}/{destination}/unload") - @ApiOperation(value = "Unload a topic") + @ApiOperation(hidden = true, value = "Unload a topic") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist") }) public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - log.info("[{}] Unloading topic {}/{}/{}/{}", clientAppId(), property, cluster, namespace, destination); - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); + validateDestinationName(property, cluster, namespace, encodedTopic); + log.info("[{}] Unloading topic {}", clientAppId(), destinationName); + + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); } - unloadTopic(dn, authoritative); + unloadTopic(destinationName, authoritative); } @GET @@ -167,13 +161,13 @@ public void unloadTopic(@PathParam("property") String property, @PathParam("clus @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public List getList(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace) { + @PathParam("namespace") String namespace) { log.info("[{}] list of topics on namespace {}/{}/{}/{}", clientAppId(), property, cluster, namespace); validateAdminAccessOnProperty(property); Policies policies = getNamespacePolicies(property, cluster, namespace); NamespaceName nsName = NamespaceName.get(property, cluster, namespace); - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { + if (!cluster.equals(Constants.GLOBAL_CLUSTER)) { validateClusterOwnership(cluster); validateClusterForProperty(property, cluster); } else { @@ -213,19 +207,19 @@ public List getList(@PathParam("property") String property, @PathParam(" } return topics; } - + @GET @Path("/{property}/{cluster}/{namespace}/{bundle}") @ApiOperation(value = "Get the list of non-persistent topics under a namespace bundle.", response = String.class, responseContainer = "List") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public List getListFromBundle(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) { + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) { log.info("[{}] list of topics on namespace bundle {}/{}/{}/{}", clientAppId(), property, cluster, namespace, bundleRange); validateAdminAccessOnProperty(property); Policies policies = getNamespacePolicies(property, cluster, namespace); - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { + if (!cluster.equals(Constants.GLOBAL_CLUSTER)) { validateClusterOwnership(cluster); validateClusterForProperty(property, cluster); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java new file mode 100644 index 0000000000000..624a3f3da6b16 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -0,0 +1,421 @@ +/** + * 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.v1; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.Encoded; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.container.AsyncResponse; +import javax.ws.rs.container.Suspended; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.policies.data.PersistentTopicStats; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +/** + */ +@Path("/persistent") +@Produces(MediaType.APPLICATION_JSON) +@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic") +public class PersistentTopics extends PersistentTopicsBase { + @GET + @Path("/{property}/{cluster}/{namespace}") + @ApiOperation(hidden = true, value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist") }) + public List getList(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + return internalGetList(); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/partitioned") + @ApiOperation(hidden = true, value = "Get the list of partitioned topics under a namespace.", response = String.class, responseContainer = "List") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist") }) + public List getPartitionedTopicList(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + return internalGetPartitionedTopicList(); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/{destination}/permissions") + @ApiOperation(hidden = true, value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the" + + "namespace level combined (union) with any eventual specific permission set on the destination.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist") }) + public Map> getPermissionsOnDestination(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetPermissionsOnDestination(); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}") + @ApiOperation(hidden = true, value = "Grant a new permission to a role on a single topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void grantPermissionsOnDestination(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role, + Set actions) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalGrantPermissionsOnDestination(role, actions); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}") + @ApiOperation(hidden = true, value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination" + + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist"), + @ApiResponse(code = 412, message = "Permissions are not set at the destination level") }) + public void revokePermissionsOnDestination(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalRevokePermissionsOnDestination(role); + } + + @PUT + @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Partitioned topic already exist") }) + public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalCreatePartitionedTopic(numPartitions, authoritative); + } + + /** + * It updates number of partitions of an existing non-global partitioned topic. It requires partitioned-topic to be + * already exist and number of new partitions must be greater than existing number of partitions. Decrementing + * number of partitions requires deletion of topic which is not supported. + * + * Already created partitioned producers and consumers can't see newly created partitions and it requires to + * recreate them at application so, newly created producers and consumers can connect to newly added partitions as + * well. Therefore, it can violate partition ordering at producers until all producers are restarted at application. + * + * @param property + * @param cluster + * @param namespace + * @param numPartitions + */ + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @ApiOperation(hidden = true, value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Partitioned topic does not exist") }) + public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + int numPartitions) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalUpdatePartitionedTopic(numPartitions); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @ApiOperation(hidden = true, value = "Get partitioned topic metadata.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetPartitionedMetadata(authoritative); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @ApiOperation(hidden = true, value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Partitioned topic does not exist") }) + public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalDeletePartitionedTopic(authoritative); + } + + @PUT + @Path("/{property}/{cluster}/{namespace}/{destination}/unload") + @ApiOperation(hidden = true, value = "Unload a topic") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalUnloadTopic(authoritative); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/{destination}") + @ApiOperation(hidden = true, value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist"), + @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") }) + public void deleteTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalDeleteTopic(authoritative); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions") + @ApiOperation(hidden = true, value = "Get the list of persistent subscriptions for a given topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public List getSubscriptions(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetSubscriptions(authoritative); + } + + @GET + @Path("{property}/{cluster}/{namespace}/{destination}/stats") + @ApiOperation(hidden = true, value = "Get the stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetStats(authoritative); + } + + @GET + @Path("{property}/{cluster}/{namespace}/{destination}/internalStats") + @ApiOperation(hidden = true, value = "Get the internal stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetInternalStats(authoritative); + } + + @GET + @Path("{property}/{cluster}/{namespace}/{destination}/internal-info") + @ApiOperation(hidden = true, value = "Get the internal stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @Suspended AsyncResponse asyncResponse) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalGetManagedLedgerInfo(asyncResponse); + } + + @GET + @Path("{property}/{cluster}/{namespace}/{destination}/partitioned-stats") + @ApiOperation(hidden = true, value = "Get the stats for the partitioned topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetPartitionedStats(authoritative); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}") + @ApiOperation(hidden = true, value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist"), + @ApiResponse(code = 412, message = "Subscription has active consumers") }) + public void deleteSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalDeleteSubscription(subName, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all") + @ApiOperation(hidden = true, value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void skipAllMessages(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalSkipAllMessages(subName, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}") + @ApiOperation(hidden = true, value = "Skip messages on a topic subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void skipMessages(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, @PathParam("numMessages") int numMessages, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalSkipMessages(subName, numMessages, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") + @ApiOperation(hidden = true, value = "Expire messages on a topic subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void expireTopicMessages(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalExpireMessages(subName, expireTimeInSeconds, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}") + @ApiOperation(hidden = true, value = "Expire messages on all subscriptions of topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void expireMessagesForAllSubscriptions(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @PathParam("expireTimeInSeconds") int expireTimeInSeconds, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}") + @ApiOperation(hidden = true, value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic/Subscription does not exist") }) + public void resetCursor(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, @PathParam("timestamp") long timestamp, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalResetCursor(subName, timestamp, authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor") + @ApiOperation(hidden = true, value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic/Subscription does not exist"), + @ApiResponse(code = 405, message = "Not supported for partitioned topics") }) + public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) { + validateDestinationName(property, cluster, namespace, encodedTopic); + internalResetCursorOnPosition(subName, authoritative, messageId); + } + + @PUT + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subscriptionName}") + @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "Creates a subscription on the topic at the specified message id") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic/Subscription does not exist"), + @ApiResponse(code = 405, message = "Not supported for partitioned topics") }) + public void createSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + @PathParam("subscriptionName") String subscriptionName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) { + validateDestinationName(property, cluster, namespace, destination); + internalCreateSubscription(subscriptionName, messageId, authoritative); + } + + @GET + @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}") + @ApiOperation(hidden = true, value = "Peek nth message on a topic subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") }) + public Response peekNthMessage(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalPeekNthMessage(subName, messagePosition, authoritative); + } + + @GET + @Path("{property}/{cluster}/{namespace}/{destination}/backlog") + @ApiOperation(hidden = true, value = "Get estimated backlog for offline topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalGetBacklog(authoritative); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{destination}/terminate") + @ApiOperation(hidden = true, value = "Terminate a topic. A topic that is terminated will not accept any more " + + "messages to be published and will let consumer to drain existing messages in backlog") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public MessageId terminate(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, cluster, namespace, encodedTopic); + return internalTerminate(authoritative); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java new file mode 100644 index 0000000000000..879a98d72ea21 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java @@ -0,0 +1,34 @@ +/** + * 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.v1; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.PropertiesBase; + +import javax.ws.rs.Consumes; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/properties") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +@Api(value = "/properties", description = "PropertiesBase admin apis", tags = "properties") +public class Properties extends PropertiesBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java new file mode 100644 index 0000000000000..fb77ee0336a75 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin.v1; + +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +import org.apache.pulsar.broker.admin.impl.ResourceQuotasBase; +import org.apache.pulsar.common.policies.data.ResourceQuota; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +@Path("/resource-quotas") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +@Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas") +public class ResourceQuotas extends ResourceQuotasBase { + + @GET + @Path("/{property}/{cluster}/{namespace}/{bundle}") + @ApiOperation(hidden = true, value = "Get resource quota of a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange) { + validateNamespaceName(property, cluster, namespace); + return internalGetNamespaceBundleResourceQuota(bundleRange); + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{bundle}") + @ApiOperation(hidden = true, value = "Set resource quota on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void setNamespaceBundleResourceQuota(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange, ResourceQuota quota) { + validateNamespaceName(property, cluster, namespace); + internalSetNamespaceBundleResourceQuota(bundleRange, quota); + } + + @DELETE + @Path("/{property}/{cluster}/{namespace}/{bundle}") + @ApiOperation(hidden = true, value = "Remove resource quota for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void removeNamespaceBundleResourceQuota(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange) { + validateNamespaceName(property, cluster, namespace); + internalRemoveNamespaceBundleResourceQuota(bundleRange); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java new file mode 100644 index 0000000000000..f0318d7e95c15 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java @@ -0,0 +1,32 @@ +/** + * 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.v2; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.BrokerStatsBase; + +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/broker-stats") +@Api(value = "/broker-stats", description = "Stats for broker", tags = "broker-stats") +@Produces(MediaType.APPLICATION_JSON) +public class BrokerStats extends BrokerStatsBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Brokers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Brokers.java new file mode 100644 index 0000000000000..7a69bb80e29b7 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Brokers.java @@ -0,0 +1,29 @@ +/** + * 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.v2; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.BrokersBase; + +import javax.ws.rs.Path; + +@Path("/brokers") +@Api(value = "/brokers", description = "BrokersBase admin apis", tags = "brokers") +public class Brokers extends BrokersBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Clusters.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Clusters.java new file mode 100644 index 0000000000000..f51dad865a4fc --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Clusters.java @@ -0,0 +1,32 @@ +/** + * 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.v2; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.ClustersBase; + +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/clusters") +@Api(value = "/clusters", description = "Cluster admin apis", tags = "clusters") +@Produces(MediaType.APPLICATION_JSON) +public class Clusters extends ClustersBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java new file mode 100644 index 0000000000000..d00499a4b8e2c --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -0,0 +1,519 @@ +/** + * 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.v2; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.MediaType; + +import org.apache.pulsar.broker.admin.impl.NamespacesBase; +import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; +import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.PersistencePolicies; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +@Path("/namespaces") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces") +public class Namespaces extends NamespacesBase { + + @GET + @Path("/{property}") + @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property doesn't exist") }) + public List getPropertyNamespaces(@PathParam("property") String property) { + return internalGetPropertyNamespaces(property); + } + + @GET + @Path("/{property}/{namespace}/topics") + @ApiOperation(value = "Get the list of all the topics under a certain namespace.", response = String.class, responseContainer = "Set") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public List getDestinations(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); + + // Validate that namespace exists, throws 404 if it doesn't exist + getNamespacePolicies(namespaceName); + + try { + return pulsar().getNamespaceService().getListOfDestinations(namespaceName); + } catch (Exception e) { + log.error("Failed to get topics list for namespace {}", namespaceName, e); + throw new RestException(e); + } + } + + @GET + @Path("/{property}/{namespace}") + @ApiOperation(value = "Get the dump all the policies specified for a namespace.", response = Policies.class) + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public Policies getPolicies(@PathParam("property") String property, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); + return getNamespacePolicies(namespaceName); + } + + @PUT + @Path("/{property}/{namespace}") + @ApiOperation(value = "Creates a new namespace with the specified policies") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster doesn't exist"), + @ApiResponse(code = 409, message = "Namespace already exists"), + @ApiResponse(code = 412, message = "Namespace name is not valid") }) + public void createNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace, + Policies policies) { + validateNamespaceName(property, namespace); + + policies = getDefaultPolicesIfNull(policies); + internalCreateNamespace(policies); + } + + @DELETE + @Path("/{property}/{namespace}") + @ApiOperation(value = "Delete a namespace and all the topics under it.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace is not empty") }) + public void deleteNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalDeleteNamespace(authoritative); + } + + @DELETE + @Path("/{property}/{namespace}/bundle/{bundle}") + @ApiOperation(value = "Delete a namespace bundle and all the topics under it.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace bundle is not empty") }) + public void deleteNamespaceBundle(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalDeleteNamespaceBundle(bundleRange, authoritative); + } + + @GET + @Path("/{property}/{namespace}/permissions") + @ApiOperation(value = "Retrieve the permissions for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Namespace is not empty") }) + public Map> getPermissions(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + return policies.auth_policies.namespace_auth; + } + + @POST + @Path("/{property}/{namespace}/permissions/{role}") + @ApiOperation(value = "Grant a new permission to a role on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void grantPermissionOnNamespace(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("role") String role, Set actions) { + validateNamespaceName(property, namespace); + internalGrantPermissionOnNamespace(role, actions); + } + + @DELETE + @Path("/{property}/{namespace}/permissions/{role}") + @ApiOperation(value = "Revoke all permissions to a role on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void revokePermissionsOnNamespace(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("role") String role) { + validateNamespaceName(property, namespace); + internalRevokePermissionsOnNamespace(role); + } + + @GET + @Path("/{property}/{namespace}/replication") + @ApiOperation(value = "Get the replication clusters for a namespace.", response = String.class, responseContainer = "List") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is not global") }) + public List getNamespaceReplicationClusters(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); + + return internalGetNamespaceReplicationClusters(); + } + + @POST + @Path("/{property}/{namespace}/replication") + @ApiOperation(value = "Set the replication clusters for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"), + @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") }) + public void setNamespaceReplicationClusters(@PathParam("property") String property, + @PathParam("namespace") String namespace, List clusterIds) { + validateNamespaceName(property, namespace); + internalSetNamespaceReplicationClusters(clusterIds); + } + + @GET + @Path("/{property}/{namespace}/messageTTL") + @ApiOperation(value = "Get the message TTL for the namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public int getNamespaceMessageTTL(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + + validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + return policies.message_ttl_in_seconds; + } + + @POST + @Path("/{property}/{namespace}/messageTTL") + @ApiOperation(value = "Set message TTL in seconds for namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Invalid TTL") }) + public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("namespace") String namespace, + int messageTTL) { + validateNamespaceName(property, namespace); + internalSetNamespaceMessageTTL(messageTTL); + } + + @POST + @Path("/{property}/{namespace}/deduplication") + @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void modifyDeduplication(@PathParam("property") String property, @PathParam("namespace") String namespace, + boolean enableDeduplication) { + validateNamespaceName(property, namespace); + internalModifyDeduplication(enableDeduplication); + } + + @GET + @Path("/{property}/{namespace}/bundles") + @ApiOperation(value = "Get the bundles split data.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") }) + public BundlesData getBundlesData(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validatePoliciesReadOnlyAccess(); + validateNamespaceName(property, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + + return policies.bundles; + } + + @PUT + @Path("/{property}/{namespace}/unload") + @ApiOperation(value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker" + + "removes all producers, consumers, and connections using this namespace, and close all destinations (including" + + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the" + + "broker completes the unloading action. This operation requires strictly super user privileges, since it would" + + "result in non-persistent message loss and unexpected connection closure to the clients.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is already unloaded or Namespace has bundles activated") }) + public void unloadNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + internalUnloadNamespace(); + } + + @PUT + @Path("/{property}/{namespace}/{bundle}/unload") + @ApiOperation(value = "Unload a namespace bundle") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void unloadNamespaceBundle(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalUnloadNamespaceBundle(bundleRange, authoritative); + } + + @PUT + @Path("/{property}/{namespace}/{bundle}/split") + @ApiOperation(value = "Split a namespace bundle") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void splitNamespaceBundle(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @QueryParam("unload") @DefaultValue("false") boolean unload) { + validateNamespaceName(property, namespace); + internalSplitNamespaceBundle(bundleRange, authoritative, unload); + } + + @POST + @Path("/{property}/{namespace}/dispatchRate") + @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void setDispatchRate(@PathParam("property") String property, @PathParam("namespace") String namespace, + DispatchRate dispatchRate) { + validateNamespaceName(property, namespace); + internalSetDispatchRate(dispatchRate); + } + + @GET + @Path("/{property}/{namespace}/dispatchRate") + @ApiOperation(value = "Get dispatch-rate configured for the namespace, -1 represents not configured yet") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public DispatchRate getDispatchRate(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetDispatchRate(); + } + + @GET + @Path("/{property}/{namespace}/backlogQuotaMap") + @ApiOperation(value = "Get backlog quota map on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public Map getBacklogQuotaMap(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); + + Policies policies = getNamespacePolicies(namespaceName); + return policies.backlog_quota_map; + } + + @POST + @Path("/{property}/{namespace}/backlogQuota") + @ApiOperation(value = " Set a backlog quota for all the destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), + @ApiResponse(code = 412, message = "Specified backlog quota exceeds retention quota. Increase retention quota and retry request") }) + public void setBacklogQuota(@PathParam("property") String property, @PathParam("namespace") String namespace, + @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, BacklogQuota backlogQuota) { + validateNamespaceName(property, namespace); + internalSetBacklogQuota(backlogQuotaType, backlogQuota); + } + + @DELETE + @Path("/{property}/{namespace}/backlogQuota") + @ApiOperation(value = "Remove a backlog quota policy from a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void removeBacklogQuota(@PathParam("property") String property, @PathParam("namespace") String namespace, + @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType) { + validateNamespaceName(property, namespace); + internalRemoveBacklogQuota(backlogQuotaType); + } + + @GET + @Path("/{property}/{namespace}/retention") + @ApiOperation(value = "Get retention config on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public RetentionPolicies getRetention(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetRetention(); + } + + @POST + @Path("/{property}/{namespace}/retention") + @ApiOperation(value = " Set retention configuration on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), + @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) + public void setRetention(@PathParam("property") String property, @PathParam("namespace") String namespace, + RetentionPolicies retention) { + validateNamespaceName(property, namespace); + internalSetRetention(retention); + } + + @POST + @Path("/{property}/{namespace}/persistence") + @ApiOperation(value = "Set the persistence configuration for all the destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), + @ApiResponse(code = 400, message = "Invalid persistence policies") }) + public void setPersistence(@PathParam("property") String property, @PathParam("namespace") String namespace, + PersistencePolicies persistence) { + validateNamespaceName(property, namespace); + internalSetPersistence(persistence); + } + + @GET + @Path("/{property}/{namespace}/persistence") + @ApiOperation(value = "Get the persistence configuration for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public PersistencePolicies getPersistence(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetPersistence(); + } + + @POST + @Path("/{property}/{namespace}/clearBacklog") + @ApiOperation(value = "Clear backlog for all destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("namespace") String namespace, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalClearNamespaceBacklog(authoritative); + } + + @POST + @Path("/{property}/{namespace}/{bundle}/clearBacklog") + @ApiOperation(value = "Clear backlog for all destinations on a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBundleBacklog(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalClearNamespaceBundleBacklog(bundleRange, authoritative); + } + + @POST + @Path("/{property}/{namespace}/clearBacklog/{subscription}") + @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBacklogForSubscription(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalClearNamespaceBacklogForSubscription(subscription, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{bundle}/clearBacklog/{subscription}") + @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalClearNamespaceBundleBacklogForSubscription(subscription, bundleRange, authoritative); + } + + @POST + @Path("/{property}/{namespace}/unsubscribe/{subscription}") + @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalUnsubscribeNamespace(subscription, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{bundle}/unsubscribe/{subscription}") + @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public void unsubscribeNamespaceBundle(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("subscription") String subscription, + @PathParam("bundle") String bundleRange, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, namespace); + internalUnsubscribeNamespaceBundle(subscription, bundleRange, authoritative); + } + + @POST + @Path("/{property}/{namespace}/subscriptionAuthMode") + @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void setSubscriptionAuthMode(@PathParam("property") String property, + @PathParam("namespace") String namespace, SubscriptionAuthMode subscriptionAuthMode) { + validateNamespaceName(property, namespace); + internalSetSubscriptionAuthMode(subscriptionAuthMode); + } + + @POST + @Path("/{property}/{namespace}/encryptionRequired") + @ApiOperation(value = "Message encryption is required or not for all topics in a namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), + @ApiResponse(code = 409, message = "Concurrent modification"), }) + public void modifyEncryptionRequired(@PathParam("property") String property, + @PathParam("namespace") String namespace, boolean encryptionRequired) { + validateNamespaceName(property, namespace); + internalModifyEncryptionRequired(encryptionRequired); + } + + private Policies getDefaultPolicesIfNull(Policies policies) { + if (policies != null) { + return policies; + } + + Policies defaultPolicies = new Policies(); + int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles(); + defaultPolicies.bundles = getBundles(defaultNumberOfBundles); + return defaultPolicies; + } + + private static final Logger log = LoggerFactory.getLogger(Namespaces.class); +} 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 new file mode 100644 index 0000000000000..55bd9be9bedc0 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -0,0 +1,159 @@ +/** + * 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.v2; + +import static com.google.common.base.Preconditions.checkNotNull; + + +import javax.ws.rs.DefaultValue; +import javax.ws.rs.Encoded; +import javax.ws.rs.GET; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response.Status; + +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.common.naming.DestinationName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.NonPersistentTopicStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +/** + */ +@Path("/non-persistent") +@Produces(MediaType.APPLICATION_JSON) +@Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic") +public class NonPersistentTopics extends PersistentTopics { + private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class); + + @GET + @Path("/{property}/{namespace}/{destination}/partitions") + @ApiOperation(value = "Get partitioned topic metadata.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return getPartitionedTopicMetadata(destinationName, authoritative); + } + + @GET + @Path("{property}/{namespace}/{destination}/stats") + @ApiOperation(value = "Get the stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public NonPersistentTopicStats getStats(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + validateAdminOperationOnDestination(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); + return ((NonPersistentTopic) topic).getStats(); + } + + @GET + @Path("{property}/{namespace}/{destination}/internalStats") + @ApiOperation(value = "Get the internal stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + validateAdminOperationOnDestination(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); + return topic.getInternalStats(); + } + + @PUT + @Path("/{property}/{namespace}/{destination}/partitions") + @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Partitioned topic already exist") }) + public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, int numPartitions, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + validateAdminAccessOnProperty(destinationName.getProperty()); + if (numPartitions <= 1) { + throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1"); + } + try { + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), + destinationName.getEncodedLocalName()); + byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); + zkCreateOptimistic(path, data); + // we wait for the data to be synced in all quorums and the observers + Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS); + log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName); + } catch (KeeperException.NodeExistsException e) { + log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName); + throw new RestException(Status.CONFLICT, "Partitioned topic already exist"); + } catch (Exception e) { + log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e); + throw new RestException(e); + } + } + + @PUT + @Path("/{property}/{namespace}/{destination}/unload") + @ApiOperation(value = "Unload a topic") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + log.info("[{}] Unloading topic {}", clientAppId(), destinationName); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + unloadTopic(destinationName, authoritative); + } + + protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) { + validateAdminAccessOnProperty(fqdn.getProperty()); + validateDestinationOwnership(fqdn, authoritative); + } + + private Topic getTopicReference(DestinationName dn) { + try { + Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString()); + checkNotNull(topic); + return topic; + } catch (Exception e) { + throw new RestException(Status.NOT_FOUND, "Topic not found"); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java new file mode 100644 index 0000000000000..b52a5add90e05 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -0,0 +1,396 @@ +/** + * 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.v2; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.Encoded; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.container.AsyncResponse; +import javax.ws.rs.container.Suspended; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.policies.data.PersistentTopicStats; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +/** + */ +@Path("/persistent") +@Produces(MediaType.APPLICATION_JSON) +@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic") +public class PersistentTopics extends PersistentTopicsBase { + + @GET + @Path("/{property}/{namespace}") + @ApiOperation(value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist") }) + public List getList(@PathParam("property") String property, @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetList(); + } + + @GET + @Path("/{property}/{namespace}/partitioned") + @ApiOperation(value = "Get the list of partitioned topics under a namespace.", response = String.class, responseContainer = "List") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist") }) + public List getPartitionedTopicList(@PathParam("property") String property, + @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetPartitionedTopicList(); + } + + @GET + @Path("/{property}/{namespace}/{destination}/permissions") + @ApiOperation(value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the" + + "namespace level combined (union) with any eventual specific permission set on the destination.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist") }) + public Map> getPermissionsOnDestination(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetPermissionsOnDestination(); + } + + @POST + @Path("/{property}/{namespace}/{destination}/permissions/{role}") + @ApiOperation(value = "Grant a new permission to a role on a single topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void grantPermissionsOnDestination(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("role") String role, Set actions) { + validateDestinationName(property, namespace, encodedTopic); + internalGrantPermissionsOnDestination(role, actions); + } + + @DELETE + @Path("/{property}/{namespace}/{destination}/permissions/{role}") + @ApiOperation(value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination" + + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace doesn't exist"), + @ApiResponse(code = 412, message = "Permissions are not set at the destination level") }) + public void revokePermissionsOnDestination(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("role") String role) { + validateDestinationName(property, namespace, encodedTopic); + internalRevokePermissionsOnDestination(role); + } + + @PUT + @Path("/{property}/{namespace}/{destination}/partitions") + @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Partitioned topic already exist") }) + public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, int numPartitions, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalCreatePartitionedTopic(numPartitions, authoritative); + } + + /** + * It updates number of partitions of an existing non-global partitioned topic. It requires partitioned-topic to be + * already exist and number of new partitions must be greater than existing number of partitions. Decrementing + * number of partitions requires deletion of topic which is not supported. + * + * Already created partitioned producers and consumers can't see newly created partitions and it requires to + * recreate them at application so, newly created producers and consumers can connect to newly added partitions as + * well. Therefore, it can violate partition ordering at producers until all producers are restarted at application. + * + * @param property + * @param cluster + * @param namespace + * @param destination + * @param numPartitions + */ + @POST + @Path("/{property}/{namespace}/{destination}/partitions") + @ApiOperation(value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Partitioned topic does not exist") }) + public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, int numPartitions) { + validateDestinationName(property, namespace, encodedTopic); + internalUpdatePartitionedTopic(numPartitions); + } + + @GET + @Path("/{property}/{namespace}/{destination}/partitions") + @ApiOperation(value = "Get partitioned topic metadata.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetPartitionedMetadata(authoritative); + } + + @DELETE + @Path("/{property}/{namespace}/{destination}/partitions") + @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Partitioned topic does not exist") }) + public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalDeletePartitionedTopic(authoritative); + } + + @PUT + @Path("/{property}/{namespace}/{destination}/unload") + @ApiOperation(value = "Unload a topic") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalUnloadTopic(authoritative); + } + + @DELETE + @Path("/{property}/{namespace}/{destination}") + @ApiOperation(value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist"), + @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") }) + public void deleteTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalDeleteTopic(authoritative); + } + + @GET + @Path("/{property}/{namespace}/{destination}/subscriptions") + @ApiOperation(value = "Get the list of persistent subscriptions for a given topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public List getSubscriptions(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetSubscriptions(authoritative); + } + + @GET + @Path("{property}/{namespace}/{destination}/stats") + @ApiOperation(value = "Get the stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PersistentTopicStats getStats(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetStats(authoritative); + } + + @GET + @Path("{property}/{namespace}/{destination}/internalStats") + @ApiOperation(value = "Get the internal stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetInternalStats(authoritative); + } + + @GET + @Path("{property}/{namespace}/{destination}/internal-info") + @ApiOperation(value = "Get the internal stats for the topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @Suspended AsyncResponse asyncResponse) { + validateDestinationName(property, namespace, encodedTopic); + internalGetManagedLedgerInfo(asyncResponse); + } + + @GET + @Path("{property}/{namespace}/{destination}/partitioned-stats") + @ApiOperation(value = "Get the stats for the partitioned topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetPartitionedStats(authoritative); + } + + @DELETE + @Path("/{property}/{namespace}/{destination}/subscription/{subName}") + @ApiOperation(value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic does not exist"), + @ApiResponse(code = 412, message = "Subscription has active consumers") }) + public void deleteSubscription(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalDeleteSubscription(subName, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip_all") + @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void skipAllMessages(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalSkipAllMessages(subName, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}") + @ApiOperation(value = "Skip messages on a topic subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void skipMessages(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @PathParam("numMessages") int numMessages, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalSkipMessages(subName, numMessages, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") + @ApiOperation(value = "Expire messages on a topic subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void expireTopicMessages(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @PathParam("expireTimeInSeconds") int expireTimeInSeconds, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalExpireMessages(subName, expireTimeInSeconds, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}") + @ApiOperation(value = "Expire messages on all subscriptions of topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic or subscription does not exist") }) + public void expireMessagesForAllSubscriptions(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("expireTimeInSeconds") int expireTimeInSeconds, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}") + @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic/Subscription does not exist") }) + public void resetCursor(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @PathParam("timestamp") long timestamp, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + internalResetCursor(subName, timestamp, authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor") + @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic/Subscription does not exist"), + @ApiResponse(code = 405, message = "Not supported for partitioned topics") }) + public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) { + validateDestinationName(property, namespace, encodedTopic); + internalResetCursorOnPosition(subName, authoritative, messageId); + } + + @GET + @Path("/{property}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}") + @ApiOperation(value = "Peek nth message on a topic subscription.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") }) + public Response peekNthMessage(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName, + @PathParam("messagePosition") int messagePosition, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalPeekNthMessage(subName, messagePosition, authoritative); + } + + @GET + @Path("{property}/{namespace}/{destination}/backlog") + @ApiOperation(value = "Get estimated backlog for offline topic.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetBacklog(authoritative); + } + + @POST + @Path("/{property}/{namespace}/{destination}/terminate") + @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more " + + "messages to be published and will let consumer to drain existing messages in backlog") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), + @ApiResponse(code = 404, message = "Topic does not exist") }) + public MessageId terminate(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateDestinationName(property, namespace, encodedTopic); + return internalTerminate(authoritative); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java new file mode 100644 index 0000000000000..19d36520f2b05 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java @@ -0,0 +1,34 @@ +/** + * 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.v2; + +import io.swagger.annotations.Api; +import org.apache.pulsar.broker.admin.impl.PropertiesBase; + +import javax.ws.rs.Consumes; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/properties") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +@Api(value = "/properties", description = "PropertiesBase admin apis", tags = "properties") +public class Properties extends PropertiesBase { +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java new file mode 100644 index 0000000000000..b84a14ff0e20f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin.v2; + +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +import org.apache.pulsar.broker.admin.impl.ResourceQuotasBase; +import org.apache.pulsar.common.policies.data.ResourceQuota; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +@Path("/resource-quotas") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +@Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas") +public class ResourceQuotas extends ResourceQuotasBase { + + @GET + @ApiOperation(value = "Get the default quota", response = String.class, responseContainer = "Set") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public ResourceQuota getDefaultResourceQuota() throws Exception { + return super.getDefaultResourceQuota(); + } + + @POST + @ApiOperation(value = "Set the default quota", response = String.class, responseContainer = "Set") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + public void setDefaultResourceQuota(ResourceQuota quota) throws Exception { + super.setDefaultResourceQuota(quota); + } + + @GET + @Path("/{property}/{namespace}/{bundle}") + @ApiOperation(value = "Get resource quota of a namespace bundle.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace does not exist") }) + public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) { + validateNamespaceName(property, namespace); + return internalGetNamespaceBundleResourceQuota(bundleRange); + } + + @POST + @Path("/{property}/{namespace}/{bundle}") + @ApiOperation(value = "Set resource quota on a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void setNamespaceBundleResourceQuota(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, ResourceQuota quota) { + validateNamespaceName(property, namespace); + internalSetNamespaceBundleResourceQuota(bundleRange, quota); + } + + @DELETE + @Path("/{property}/{namespace}/{bundle}") + @ApiOperation(value = "Remove resource quota for a namespace.") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 409, message = "Concurrent modification") }) + public void removeNamespaceBundleResourceQuota(@PathParam("property") String property, + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) { + validateNamespaceName(property, namespace); + internalRemoveNamespaceBundleResourceQuota(bundleRange); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java index 19d3f348e8bd9..80cb31cb100c6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java @@ -253,7 +253,7 @@ public static boolean isLoadSheddingEnabled(final PulsarService pulsar) { * @param assignedBundleName * Name of bundle to be assigned. * @param candidates - * Brokers available for placement. + * BrokersBase available for placement. * @param brokerToNamespaceToBundleRange * Map from brokers to namespaces to bundle ranges. */ @@ -302,14 +302,14 @@ public static void removeMostServicingBrokersForNamespace(final String assignedB * eg. *
      * Before:
-     * Domain-count  Brokers-count
+     * Domain-count  BrokersBase-count
      * ____________  ____________
      * d1-3          b1-2,b2-1
      * d2-3          b3-2,b4-1
      * d3-4          b5-2,b6-2
      *
      * After filtering: "candidates" brokers
-     * Domain-count  Brokers-count
+     * Domain-count  BrokersBase-count
      * ____________  ____________
      * d1-3          b2-1
      * d2-3          b4-1
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
index 0393d6a36841f..300bd84e97e70 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
@@ -52,6 +52,7 @@
 import org.apache.pulsar.common.naming.DestinationDomain;
 import org.apache.pulsar.common.naming.DestinationName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
+import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.util.Codec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -211,7 +212,8 @@ public static CompletableFuture lookupDestinationAsync(PulsarService pu
             if (differentClusterData != null) {
                 if (log.isDebugEnabled()) {
                     log.debug("[{}] Redirecting the lookup call to {}/{} cluster={}", clientAppId,
-                            differentClusterData.getBrokerServiceUrl(), differentClusterData.getBrokerServiceUrlTls(), cluster);
+                            differentClusterData.getBrokerServiceUrl(), differentClusterData.getBrokerServiceUrlTls(),
+                            cluster);
                 }
                 validationFuture.complete(newLookupResponse(differentClusterData.getBrokerServiceUrl(),
                         differentClusterData.getBrokerServiceUrlTls(), true, LookupType.Redirect, requestId, false));
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 e7b885a427190..5319344b4f4d5 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
@@ -726,16 +726,18 @@ public ServiceUnitId getServiceUnitId(DestinationName destinationName) throws Ex
         return getBundle(destinationName);
     }
 
-    public List getListOfDestinations(String property, String cluster, String namespace) throws Exception {
+    public List getListOfDestinations(NamespaceName namespaceName) throws Exception {
         List destinations = Lists.newArrayList();
 
         // For every topic there will be a managed ledger created.
         try {
-            String path = String.format("/managed-ledgers/%s/%s/%s/persistent", property, cluster, namespace);
-            LOG.debug("Getting children from managed-ledgers now: {}", path);
+            String path = String.format("/managed-ledgers/%s/persistent", namespaceName);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Getting children from managed-ledgers now: {}", path);
+            }
+
             for (String destination : pulsar.getLocalZkCacheService().managedLedgerListCache().get(path)) {
-                destinations.add(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace,
-                        Codec.decode(destination)));
+                destinations.add(String.format("persistent://%s/%s", namespaceName, Codec.decode(destination)));
             }
         } catch (KeeperException.NoNodeException e) {
             // NoNode means there are no persistent topics for this namespace
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitZkUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitZkUtils.java
index 210b335079936..a925857890fb4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitZkUtils.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitZkUtils.java
@@ -75,10 +75,17 @@ public static final NamespaceBundle suBundleFromPath(String path, NamespaceBundl
         String[] parts = path.split("/");
         checkArgument(parts.length > 2);
         checkArgument(parts[1].equals("namespace"));
-        checkArgument(parts.length > 5);
-
-        Range range = getHashRange(parts[5]);
-        return factory.getBundle(NamespaceName.get(parts[2], parts[3], parts[4]), range);
+        checkArgument(parts.length > 4);
+
+        if (parts.length > 5) {
+            // this is a V1 path prop/cluster/namespace/hash
+            Range range = getHashRange(parts[5]);
+            return factory.getBundle(NamespaceName.get(parts[2], parts[3], parts[4]), range);
+        } else {
+            // this is a V2 path prop/namespace/hash
+            Range range = getHashRange(parts[4]);
+            return factory.getBundle(NamespaceName.get(parts[2], parts[3]), range);
+        }
     }
 
     private static Range getHashRange(String rangePathPart) {
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 2f3b27a75bae6..1eff83b0efaf9 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
@@ -638,7 +638,7 @@ public CompletableFuture getManagedLedgerConfig(Destination
             try {
                 policies = pulsar
                         .getConfigurationCache().policiesCache().get(AdminResource.path(POLICIES,
-                                namespace.getProperty(), namespace.getCluster(), namespace.getLocalName()));
+                                namespace.toString()));
             } catch (Throwable t) {
                 // Ignoring since if we don't have policies, we fallback on the default
                 log.warn("Got exception when reading persistence policy for {}: {}", topicName, t.getMessage(), t);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
index b50eee6ec6255..0b8d512808de1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
@@ -20,7 +20,7 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.commons.lang3.StringUtils.isNotBlank;
-import static org.apache.pulsar.broker.admin.PersistentTopics.getPartitionedTopicMetadata;
+import static org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.getPartitionedTopicMetadata;
 import static org.apache.pulsar.broker.lookup.DestinationLookup.lookupDestinationAsync;
 import static org.apache.pulsar.common.api.Commands.newLookupErrorResponse;
 import static org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion.v5;
@@ -278,7 +278,7 @@ protected void handleLookup(CommandLookupTopic lookup) {
                 return null;
             }).exceptionally(ex -> {
                 final String msg = "Exception occured while trying to authorize lookup";
-                log.warn("[{}] {} with role {} on topic {}", remoteAddress, msg, authRole, topicName);
+                log.warn("[{}] {} with role {} on topic {}", remoteAddress, msg, authRole, topicName, ex);
                 ctx.writeAndFlush(newLookupErrorResponse(ServerError.AuthorizationError, msg, requestId));
                 lookupSemaphore.release();
                 return null;
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 c17182c542209..60031fcbd0d32 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
@@ -45,14 +45,10 @@
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.admin.AdminResource;
-import org.apache.pulsar.broker.admin.Namespaces;
 import org.apache.pulsar.broker.authentication.AuthenticationDataHttps;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
 import org.apache.pulsar.broker.namespace.NamespaceService;
-import org.apache.pulsar.common.naming.DestinationName;
-import org.apache.pulsar.common.naming.NamespaceBundle;
-import org.apache.pulsar.common.naming.NamespaceBundles;
-import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.*;
 import org.apache.pulsar.common.policies.data.BundlesData;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.Policies;
@@ -277,12 +273,13 @@ private URI getRedirectionUrl(ClusterData differentClusterData) throws Malformed
     }
 
     protected static CompletableFuture getClusterDataIfDifferentCluster(PulsarService pulsar,
-            String cluster, String clientAppId) {
+         String cluster, String clientAppId) {
 
         CompletableFuture clusterDataFuture = new CompletableFuture<>();
 
         if (!isValidCluster(pulsar, cluster)) {
             try {
+                // this code should only happen with a v1 namespace format prop/cluster/namespaces
                 if (!pulsar.getConfiguration().getClusterName().equals(cluster)) {
                     // redirect to the cluster requested
                     pulsar.getConfigurationCache().clustersCache().getAsync(path("clusters", cluster))
@@ -310,15 +307,15 @@ protected static CompletableFuture getClusterDataIfDifferentCluster
         return clusterDataFuture;
     }
 
-    protected static boolean isValidCluster(PulsarService pulsarSevice, String cluster) {// If the cluster name is
-        // "global", don't validate the
-        // cluster ownership.
+    static boolean isValidCluster(PulsarService pulsarService, String cluster) {// If the cluster name is
+        // cluster == null or "global", don't validate the
+        // cluster ownership. Cluster will be null in v2 naming.
         // The validation will be done by checking the namespace configuration
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
+        if (cluster == null || Constants.GLOBAL_CLUSTER.equals(cluster)) {
             return true;
         }
 
-        if (!pulsarSevice.getConfiguration().isAuthorizationEnabled()) {
+        if (!pulsarService.getConfiguration().isAuthorizationEnabled()) {
             // Without authorization, any cluster name should be valid and accepted by the broker
             return true;
         }
@@ -565,8 +562,7 @@ protected static CompletableFuture checkLocalOrGetPeerReplicationCl
         }
         final CompletableFuture validationFuture = new CompletableFuture<>();
         final String localCluster = pulsarService.getConfiguration().getClusterName();
-        final String path = AdminResource.path(POLICIES, namespace.getProperty(), namespace.getCluster(),
-                namespace.getLocalName());
+        final String path = AdminResource.path(POLICIES, namespace.toString());
 
         pulsarService.getConfigurationCache().policiesCache().getAsync(path).thenAccept(policiesResult -> {
             if (policiesResult.isPresent()) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java
index 27e3992b07c42..fd4d7ed9cfa59 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java
@@ -70,11 +70,11 @@ public static String fixVersionString(String version) {
     }
 
     /**
-     * Looks for a resource in the jar which is expected to be a java.util.Properties, then
+     * Looks for a resource in the jar which is expected to be a java.util.PropertiesBase, then
      * extract a specific property value.
      *
      * @return the property value, or null if the resource does not exist or the resource
-     *         is not a valid java.util.Properties or the resource does not contain the
+     *         is not a valid java.util.PropertiesBase or the resource does not contain the
      *         named property
      */
     private static String getPropertyFromResource(String resource, String propertyName) {
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 444d224a34063..503811633bef6 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
@@ -1493,7 +1493,7 @@ class CustomPropertyAdmin extends PropertyAdmin {
 
     /**
      * 
-     * Verify: PersistentTopics.expireMessages()/expireMessagesForAllSubscriptions()
+     * Verify: PersistentTopicsBase.expireMessages()/expireMessagesForAllSubscriptions()
      * 1. Created multiple shared subscriptions and publisher on topic
      * 2. Publish messages on the topic
      * 3. expire message on sub-1 : backlog for sub-1 must be 0
@@ -1553,7 +1553,7 @@ public void testPersistentTopicsExpireMessages() throws Exception {
     }
 
     /**
-     * Verify: PersistentTopics.expireMessages()/expireMessagesForAllSubscriptions() for PartitionTopic
+     * Verify: PersistentTopicsBase.expireMessages()/expireMessagesForAllSubscriptions() for PartitionTopic
      *
      * @throws Exception
      */
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
index c587033af5d18..52d1186473f40 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
@@ -48,11 +48,18 @@
 
 import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats;
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.pulsar.broker.admin.v1.BrokerStats;
+import org.apache.pulsar.broker.admin.v1.Brokers;
+import org.apache.pulsar.broker.admin.v1.Clusters;
+import org.apache.pulsar.broker.admin.v1.Properties;
+import org.apache.pulsar.broker.admin.v1.Namespaces;
+import org.apache.pulsar.broker.admin.v1.PersistentTopics;
+import org.apache.pulsar.broker.admin.v1.ResourceQuotas;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.broker.loadbalance.ResourceUnit;
 import org.apache.pulsar.broker.web.PulsarWebResource;
 import org.apache.pulsar.broker.web.RestException;
+import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData;
 import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType;
@@ -90,7 +97,6 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
     private BrokerStats brokerStats;
 
     private Field uriField;
-    private UriInfo uriInfo;
     private final String configClusterName = "use";
 
     public AdminTest() {
@@ -146,7 +152,6 @@ public void setup() throws Exception {
 
         uriField = PulsarWebResource.class.getDeclaredField("uri");
         uriField.setAccessible(true);
-        uriInfo = mock(UriInfo.class);
 
         persistentTopics = spy(new PersistentTopics());
         persistentTopics.setServletContext(new MockServletContext());
@@ -511,7 +516,7 @@ void resourceQuotas() throws Exception {
         String namespace = "ns";
         String bundleRange = "0x00000000_0xffffffff";
         Policies policies = new Policies();
-        doReturn(policies).when(resourceQuotas).getNamespacePolicies(property, cluster, namespace);
+        doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace));
         doReturn("client-id").when(resourceQuotas).clientAppId();
 
         try {
@@ -570,8 +575,7 @@ void brokerStats() throws Exception {
         StreamingOutput destination = brokerStats.getDestinations2();
         assertNotNull(destination);
         try {
-            Map> resource = brokerStats.getBrokerResourceAvailability("prop", "use",
-                    "ns2");
+            brokerStats.getBrokerResourceAvailability("prop", "use", "ns2");
             fail("should have failed as ModularLoadManager doesn't support it");
         } catch (RestException re) {
             // Ok
@@ -586,13 +590,12 @@ void persistentTopics() throws Exception {
         final String namespace = "ns";
         final String destination = "ds1";
         Policies policies = new Policies();
-        doReturn(policies).when(resourceQuotas).getNamespacePolicies(property, cluster, namespace);
+        doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace));
         doReturn("client-id").when(resourceQuotas).clientAppId();
         // create policies
         PropertyAdmin admin = new PropertyAdmin();
         admin.getAllowedClusters().add(cluster);
-        ZkUtils.createFullPathOptimistic(mockZookKeeper,
-                PulsarWebResource.path(POLICIES, property, cluster, namespace),
+        ZkUtils.createFullPathOptimistic(mockZookKeeper, PulsarWebResource.path(POLICIES, property, cluster, namespace),
                 ObjectMapperFactory.getThreadLocal().writeValueAsBytes(new Policies()), ZooDefs.Ids.OPEN_ACL_UNSAFE,
                 CreateMode.PERSISTENT);
 
@@ -601,8 +604,8 @@ void persistentTopics() throws Exception {
         // create destination
         assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList());
         persistentTopics.createPartitionedTopic(property, cluster, namespace, destination, 5, false);
-        assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList(
-                String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, destination)));
+        assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists
+                .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, destination)));
 
         CountDownLatch notificationLatch = new CountDownLatch(2);
         configurationCache.policiesCache().registerListener((path, data, stat) -> {
@@ -635,5 +638,5 @@ public void testRestExceptionMessage() {
         assertEquals(exception.getMessage(), message);
 
     }
-    
+
 }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
index 36c9ec2b75f9a..f9ff652375419 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
@@ -47,6 +47,8 @@
 import javax.ws.rs.core.UriInfo;
 
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.pulsar.broker.admin.v1.Namespaces;
+import org.apache.pulsar.broker.admin.v1.PersistentTopics;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.broker.namespace.NamespaceEphemeralData;
 import org.apache.pulsar.broker.namespace.NamespaceService;
@@ -1064,20 +1066,16 @@ public void testValidateDestinationOwnership() throws Exception {
             doReturn(false).when(topics).isRequestHttps();
             doReturn("test").when(topics).clientAppId();
             mockWebUrl(localWebServiceUrl, testNs);
+            doReturn("persistent").when(topics).domain();
 
             try {
-                topics.validateAdminOperationOnDestination(topicName, false);
+                topics.validateDestinationName(topicName.getProperty(), topicName.getCluster(),
+                        topicName.getNamespacePortion(), topicName.getEncodedLocalName());
+                topics.validateAdminOperationOnDestination(false);
             } catch (RestException e) {
                 fail("validateAdminAccessOnProperty failed");
             }
 
-            try {
-                topics.validateAdminOperationOnDestination(DestinationName.get(""), false);
-                fail("validateAdminAccessOnProperty failed");
-            } catch (Exception e) {
-                // OK
-            }
-
         } catch (RestException e) {
             fail("validateAdminAccessOnProperty failed");
         }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java
new file mode 100644
index 0000000000000..aed20fa556701
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.common.naming;
+
+public class Constants {
+
+    public static final String GLOBAL_CLUSTER = "global";
+
+    private Constants() {}
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/DestinationName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/DestinationName.java
index 3505010710606..ef40ced3245e9 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/DestinationName.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/DestinationName.java
@@ -18,8 +18,6 @@
  */
 package org.apache.pulsar.common.naming;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -64,6 +62,16 @@ public DestinationName load(String name) throws Exception {
                 }
             });
 
+    public static DestinationName get(String domain, NamespaceName namespaceName, String destination) {
+        String name = domain + "://" + namespaceName.toString() + '/' + destination;
+        return DestinationName.get(name);
+    }
+
+    public static DestinationName get(String domain, String property, String namespace, String destination) {
+        String name = domain + "://" + property + '/' + namespace + '/' + destination;
+        return DestinationName.get(name);
+    }
+
     public static DestinationName get(String domain, String property, String cluster, String namespace,
             String destination) {
         String name = domain + "://" + property + '/' + cluster + '/' + namespace + '/' + destination;
@@ -92,7 +100,9 @@ public static boolean isValid(String destination) {
     private DestinationName(String destination) {
         this.destination = destination;
         try {
-            // persistent://property/cluster/namespace/topic
+            // The topic name can be in two different forms:
+            // new:    persistent://property/namespace/topic
+            // legacy: persistent://property/cluster/namespace/topic
             if (!destination.contains("://")) {
                 throw new IllegalArgumentException(
                         "Invalid destination name: " + destination + " -- Domain is missing");
@@ -102,29 +112,44 @@ private DestinationName(String destination) {
             this.domain = DestinationDomain.getEnum(parts.get(0));
 
             String rest = parts.get(1);
-            // property/cluster/namespace/
+
+            // The rest of the name can be in different forms:
+            // new:    property/namespace/
+            // legacy: property/cluster/namespace/
             // Examples of localName:
             // 1. some/name/xyz//
             // 2. /xyz-123/feeder-2
+
+
             parts = Splitter.on("/").limit(4).splitToList(rest);
-            if (parts.size() != 4) {
+            if (parts.size() == 3) {
+                // New topic name without cluster name
+                this.property = parts.get(0);
+                this.cluster = null;
+                this.namespacePortion = parts.get(1);
+                this.localName = parts.get(2);
+                this.partitionIndex = getPartitionIndex(destination);
+                this.namespaceName = NamespaceName.get(property, namespacePortion);
+            } else if (parts.size() == 4) {
+                // Legacy topic name that includes cluster name
+                this.property = parts.get(0);
+                this.cluster = parts.get(1);
+                this.namespacePortion = parts.get(2);
+                this.localName = parts.get(3);
+                this.partitionIndex = getPartitionIndex(destination);
+                this.namespaceName = NamespaceName.get(property, cluster, namespacePortion);
+            } else {
                 throw new IllegalArgumentException("Invalid destination name: " + destination);
             }
 
-            this.property = parts.get(0);
-            this.cluster = parts.get(1);
-            this.namespacePortion = parts.get(2);
-            this.localName = parts.get(3);
-            this.partitionIndex = getPartitionIndex(destination);
 
-            NamespaceName.validateNamespaceName(property, cluster, namespacePortion);
-            if (checkNotNull(localName).isEmpty()) {
+            if (localName == null || localName.isEmpty()) {
                 throw new IllegalArgumentException("Invalid destination name: " + destination);
             }
         } catch (NullPointerException e) {
             throw new IllegalArgumentException("Invalid destination name: " + destination, e);
         }
-        namespaceName = NamespaceName.get(property, cluster, namespacePortion);
+
     }
 
     /**
@@ -156,6 +181,7 @@ public String getProperty() {
         return property;
     }
 
+    @Deprecated
     public String getCluster() {
         return cluster;
     }
@@ -229,9 +255,16 @@ public static int getPartitionIndex(String topic) {
      * @return the relative path to be used in persistence
      */
     public String getPersistenceNamingEncoding() {
-        // The convention is: domain://property/cluster/namespace/destination
-        // We want to persist in the order: property/cluster/namespace/domain/destination
-        return String.format("%s/%s/%s/%s/%s", property, cluster, namespacePortion, domain, getEncodedLocalName());
+        // The convention is: domain://property/namespace/topic
+        // We want to persist in the order: property/namespace/domain/topic
+
+        // For legacy naming scheme, the convention is: domain://property/cluster/namespace/topic
+        // We want to persist in the order: property/cluster/namespace/domain/topic
+        if (cluster == null) {
+            return String.format("%s/%s/%s/%s", property, namespacePortion, domain, getEncodedLocalName());
+        } else {
+            return String.format("%s/%s/%s/%s/%s", property, cluster, namespacePortion, domain, getEncodedLocalName());
+        }
     }
 
     /**
@@ -244,11 +277,15 @@ public String getPersistenceNamingEncoding() {
      * @return
      */
     public String getLookupName() {
-        return String.format("%s/%s/%s/%s/%s", domain, property, cluster, namespacePortion, getEncodedLocalName());
+        if (cluster == null) {
+            return String.format("%s/%s/%s/%s", domain, property, namespacePortion, getEncodedLocalName());
+        } else {
+            return String.format("%s/%s/%s/%s/%s", domain, property, cluster, namespacePortion, getEncodedLocalName());
+        }
     }
 
     public boolean isGlobal() {
-        return "global".equals(cluster);
+        return cluster == null || Constants.GLOBAL_CLUSTER.equalsIgnoreCase(cluster);
     }
 
     @Override
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java
index a179f9af54ca1..8ff474da8fcff 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java
@@ -20,8 +20,6 @@
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
@@ -35,10 +33,10 @@ public class NamespaceName implements ServiceUnitId {
 
     private final String namespace;
 
-    private String property;
-    private String cluster;
-    private String localName;
-    
+    private final String property;
+    private final String cluster;
+    private final String localName;
+
     private static final LoadingCache cache = CacheBuilder.newBuilder().maximumSize(100000)
             .expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader() {
                 @Override
@@ -47,6 +45,11 @@ public NamespaceName load(String name) throws Exception {
                 }
             });
 
+    public static NamespaceName get(String property, String namespace) {
+        validateNamespaceName(property, namespace);
+        return get(property + '/' + namespace);
+    }
+
     public static NamespaceName get(String property, String cluster, String namespace) {
         validateNamespaceName(property, cluster, namespace);
         return get(property + '/' + cluster + '/' + namespace);
@@ -68,14 +71,37 @@ public static NamespaceName get(String namespace) {
     }
 
     private NamespaceName(String namespace) {
-        try {
-            checkNotNull(namespace);
-        } catch (NullPointerException e) {
+        if (namespace == null || namespace.isEmpty()) {
             throw new IllegalArgumentException("Invalid null namespace: " + namespace);
         }
 
         // Verify it's a proper namespace
-        validateNamespaceName(namespace);
+        // The namespace name is composed of /
+        // or in the legacy format with the cluster name:
+        // //
+        try {
+
+            String[] parts = namespace.split("/");
+            if (parts.length == 2) {
+                // New style namespace : /
+                validateNamespaceName(parts[0], parts[1]);
+
+                property = parts[0];
+                cluster = null;
+                localName = parts[1];
+            } else if (parts.length == 3) {
+                // Old style namespace: //
+                validateNamespaceName(parts[0], parts[1], parts[2]);
+
+                property = parts[0];
+                cluster = parts[1];
+                localName = parts[2];
+            } else {
+                throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace);
+            }
+        } catch (NullPointerException e) {
+            throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace, e);
+        }
         this.namespace = namespace;
     }
 
@@ -83,6 +109,7 @@ public String getProperty() {
         return property;
     }
 
+    @Deprecated
     public String getCluster() {
         return cluster;
     }
@@ -92,7 +119,7 @@ public String getLocalName() {
     }
 
     public boolean isGlobal() {
-        return "global".equals(cluster);
+        return cluster == null || Constants.GLOBAL_CLUSTER.equalsIgnoreCase(cluster);
     }
 
     public String getPersistentTopicName(String localTopic) {
@@ -136,46 +163,37 @@ public int hashCode() {
         return namespace.hashCode();
     }
 
-    public static void validateNamespaceName(String property, String cluster, String namespace) {
+    public static void validateNamespaceName(String property, String namespace) {
         try {
             checkNotNull(property);
-            checkNotNull(cluster);
             checkNotNull(namespace);
-            if (property.isEmpty() || cluster.isEmpty() || namespace.isEmpty()) {
+            if (property.isEmpty() || namespace.isEmpty()) {
                 throw new IllegalArgumentException(
-                        String.format("Invalid namespace format. namespace: %s/%s/%s", property, cluster, namespace));
+                        String.format("Invalid namespace format. namespace: %s/%s", property, namespace));
             }
             NamedEntity.checkName(property);
-            NamedEntity.checkName(cluster);
             NamedEntity.checkName(namespace);
         } catch (NullPointerException e) {
             throw new IllegalArgumentException(
-                    String.format("Invalid namespace format. namespace: %s/%s/%s", property, cluster, namespace), e);
+                    String.format("Invalid namespace format. namespace: %s/%s/%s", property, namespace), e);
         }
     }
 
-    private void validateNamespaceName(String namespace) {
-        // assume the namespace is in the form of //
+    public static void validateNamespaceName(String property, String cluster, String namespace) {
         try {
+            checkNotNull(property);
+            checkNotNull(cluster);
             checkNotNull(namespace);
-            String testUrl = String.format("http://%s", namespace);
-            URI uri = new URI(testUrl);
-            checkNotNull(uri.getPath());
-            NamedEntity.checkURI(uri, testUrl);
-
-            String[] parts = uri.getPath().split("/");
-            if (parts.length != 3) {
-                throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace);
+            if (property.isEmpty() || cluster.isEmpty() || namespace.isEmpty()) {
+                throw new IllegalArgumentException(
+                        String.format("Invalid namespace format. namespace: %s/%s/%s", property, cluster, namespace));
             }
-            validateNamespaceName(uri.getHost(), parts[1], parts[2]);
-
-            property = uri.getHost();
-            cluster = parts[1];
-            localName = parts[2];
-        } catch (URISyntaxException e) {
-            throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace, e);
+            NamedEntity.checkName(property);
+            NamedEntity.checkName(cluster);
+            NamedEntity.checkName(namespace);
         } catch (NullPointerException e) {
-            throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace, e);
+            throw new IllegalArgumentException(
+                    String.format("Invalid namespace format. namespace: %s/%s/%s", property, cluster, namespace), e);
         }
     }
 
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/DestinationNameTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/DestinationNameTest.java
index 74574d84eaa12..caa16fad3b205 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/DestinationNameTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/DestinationNameTest.java
@@ -118,13 +118,6 @@ void destination() {
             // Ok
         }
 
-        try {
-            DestinationName.get("persistent://property/cluster/namespace");
-            fail("Should have raised exception");
-        } catch (IllegalArgumentException e) {
-            // Ok
-        }
-
         try {
             DestinationName.get("property/cluster/namespace/destination");
             fail("Should have raised exception");
@@ -223,4 +216,28 @@ public void testDecodeEncode() throws Exception {
         assertEquals(name.getEncodedLocalName(), encodedName);
         assertEquals(name.getPersistenceNamingEncoding(), "prop/colo/ns/persistent/" + encodedName);
     }
+
+    @Test
+    public void testTopicNameWithoutCluster() throws Exception {
+        DestinationName dn = DestinationName.get("persistent://property/namespace/destination");
+
+        assertEquals(dn.getNamespace(), "property/namespace");
+
+        assertEquals(dn, DestinationName.get("persistent", "property", "namespace", "destination"));
+
+        assertEquals(dn.hashCode(),
+                DestinationName.get("persistent", "property", "namespace", "destination").hashCode());
+
+        assertEquals(dn.toString(), "persistent://property/namespace/destination");
+        assertEquals(dn.getDomain(), DestinationDomain.persistent);
+        assertEquals(dn.getProperty(), "property");
+        assertEquals(dn.getCluster(), null);
+        assertEquals(dn.getNamespacePortion(), "namespace");
+        assertEquals(dn.getNamespace(), "property/namespace");
+        assertEquals(dn.getLocalName(), "destination");
+
+        assertEquals(dn.getEncodedLocalName(), "destination");
+        assertEquals(dn.getPartitionedTopicName(), "persistent://property/namespace/destination");
+        assertEquals(dn.getPersistenceNamingEncoding(), "property/namespace/persistent/destination");
+    }
 }
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/NamespaceNameTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/NamespaceNameTest.java
index 7d6a7cf0b3c20..8d13c2473e92d 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/NamespaceNameTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/NamespaceNameTest.java
@@ -59,13 +59,6 @@ void namespace() {
             // Ok
         }
 
-        try {
-            NamespaceName.get("property/namespace");
-            fail("Should have raised exception");
-        } catch (IllegalArgumentException e) {
-            // Ok
-        }
-
         try {
             NamespaceName.get("property/cluster/namespace/destination");
             fail("Should have raised exception");
@@ -127,13 +120,6 @@ void namespace() {
             // Ok
         }
 
-        try {
-            NamespaceName.get("_pulsar/cluster/namespace");
-            fail("Should have raised exception");
-        } catch (IllegalArgumentException e) {
-            // Ok
-        }
-
         try {
             NamespaceName.get(null, "cluster", "namespace");
             fail("Should have raised exception");
@@ -176,13 +162,6 @@ void namespace() {
             // Ok
         }
 
-        try {
-            NamespaceName.get("pulsar/cluster/");
-            fail("Should have raised exception");
-        } catch (IllegalArgumentException e) {
-            // Ok
-        }
-
         try {
             NamespaceName.get("pulsar", "cluster", null);
             fail("Should have raised exception");
@@ -202,4 +181,14 @@ void namespace() {
         assertEquals(v2Namespace.getCluster(), "colo1");
         assertEquals(v2Namespace.getLocalName(), "testns-1");
     }
+
+    @Test
+    void testNewScheme() {
+        NamespaceName ns = NamespaceName.get("my-tenant/my-namespace");
+        assertEquals(ns.getProperty(), "my-tenant");
+        assertEquals(ns.getLocalName(), "my-namespace");
+        assertEquals(ns.isGlobal(), true);
+        assertEquals(ns.getCluster(), null);
+        assertEquals(ns.getPersistentTopicName("my-topic"), "persistent://my-tenant/my-namespace/my-topic");
+    }
 }