From 73db2ca28f30742dfc65db6db6ef4810f52f299e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Sun, 7 Jan 2018 20:36:44 +0100 Subject: [PATCH 01/10] Make DestinationName and NamespaceName classes to handle both formats --- .../pulsar/common/naming/DestinationName.java | 7 +- .../pulsar/common/naming/NamespaceName.java | 86 ++++++++++++------- .../common/naming/DestinationNameTest.java | 31 +++++++ .../common/naming/NamespaceNameTest.java | 31 +++---- 4 files changed, 100 insertions(+), 55 deletions(-) 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..7e24e369d5de8 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 @@ -64,6 +64,11 @@ public DestinationName load(String name) throws Exception { } }); + 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; @@ -118,7 +123,7 @@ private DestinationName(String destination) { 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) { 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..ea1d1610caf71 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 @@ -35,10 +35,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 +47,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 +73,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 +111,7 @@ public String getProperty() { return property; } + @Deprecated public String getCluster() { return cluster; } @@ -92,7 +121,7 @@ public String getLocalName() { } public boolean isGlobal() { - return "global".equals(cluster); + return cluster == null || "global".equals(cluster); } public String getPersistentTopicName(String localTopic) { @@ -136,46 +165,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..34b0078214974 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 @@ -223,4 +223,35 @@ public void testDecodeEncode() throws Exception { assertEquals(name.getEncodedLocalName(), encodedName); assertEquals(name.getPersistenceNamingEncoding(), "prop/colo/ns/persistent/" + encodedName); } + + @Test + public void testTopicNameWithoutCluster() throws Exception { + assertEquals(DestinationName.get("persistent://property/namespace/destination").getNamespace(), + "property/namespace"); + + assertEquals(DestinationName.get("persistent://property/namespace/destination"), + DestinationName.get("persistent", "property", "namespace", "destination")); + + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").hashCode(), + DestinationName.get("persistent", "property", "cluster", "namespace", "destination").hashCode()); + + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").toString(), + "persistent://property/cluster/namespace/destination"); + + assertFalse(DestinationName.get("persistent://property/cluster/namespace/destination") + .equals("persistent://property/cluster/namespace/destination")); + + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getDomain(), + DestinationDomain.persistent); + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getProperty(), + "property"); + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getCluster(), + "cluster"); + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getNamespacePortion(), + "namespace"); + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getNamespace(), + "property/cluster/namespace"); + assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getLocalName(), + "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"); + } } From 2956a2eb36cb972a39c31d11ebe0207986f28f63 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 9 Jan 2018 13:06:03 +0100 Subject: [PATCH 02/10] Refactor /namespaces and /resource-quotas handlers for new-style namespace names --- .../apache/pulsar/broker/PulsarService.java | 3 +- .../pulsar/broker/admin/AdminResource.java | 20 +- .../pulsar/broker/admin/Namespaces.java | 1448 ++--------------- .../pulsar/broker/admin/NamespacesLegacy.java | 562 +++++++ .../pulsar/broker/admin/ResourceQuotas.java | 122 +- .../broker/admin/ResourceQuotasLegacy.java | 79 + .../broker/admin/impl/NamespacesBase.java | 1259 ++++++++++++++ .../broker/admin/impl/ResourceQuotasBase.java | 135 ++ .../broker/namespace/NamespaceService.java | 16 +- .../apache/pulsar/broker/admin/AdminTest.java | 28 +- .../pulsar/broker/admin/NamespacesTest.java | 4 +- .../pulsar/common/naming/DestinationName.java | 58 +- .../pulsar/common/naming/NamespaceName.java | 2 - .../common/naming/DestinationNameTest.java | 44 +- 14 files changed, 2322 insertions(+), 1458 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NamespacesLegacy.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotasLegacy.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java 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 2d7f99b6f50ba..68590bbe1660e 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 @@ -462,8 +462,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 f542a10f03c1a..5ca8c574f3cea 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 @@ -224,20 +224,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); } } @@ -246,11 +246,11 @@ 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(); } @@ -258,7 +258,7 @@ ZooKeeperDataCache localPoliciesCache() { return pulsar().getLocalZkCacheService().policiesCache(); } - ZooKeeperDataCache clustersCache() { + protected ZooKeeperDataCache clustersCache() { return pulsar().getConfigurationCache().clustersCache(); } @@ -266,7 +266,7 @@ ZooKeeperChildrenCache managedLedgerListCache() { return pulsar().getLocalZkCacheService().managedLedgerListCache(); } - Set clusters() { + protected Set clusters() { try { return pulsar().getConfigurationCache().clustersListCache().get(); } catch (Exception e) { @@ -294,7 +294,7 @@ protected PartitionedTopicMetadata getPartitionedTopicMetadata(String property, // serve/redirect request else fail partitioned-metadata-request so, client fails while creating // producer/consumer validateGlobalNamespaceOwnership(dn.getNamespaceObject()); - + try { checkConnect(dn); } catch (WebApplicationException e) { @@ -354,5 +354,5 @@ public PartitionedTopicMetadata deserialize(String key, byte[] content) throws E } return metadataFuture; } - + } 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 index cb8cf72a8dcd4..652f208e8551b 100644 --- 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 @@ -18,21 +18,9 @@ */ package org.apache.pulsar.broker.admin; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; - -import java.net.URI; -import java.net.URL; -import java.util.LinkedHashSet; 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 javax.ws.rs.Consumes; import javax.ws.rs.DELETE; @@ -44,63 +32,34 @@ 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.admin.impl.NamespacesBase; import org.apache.pulsar.broker.web.RestException; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.util.FutureUtil; -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.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; -import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; - @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 class Namespaces extends NamespacesBase { public static final String GLOBAL_CLUSTER = "global"; - private static final long MAX_BUNDLES = ((long) 1) << 32; @GET @Path("/{property}") @@ -108,327 +67,109 @@ public class Namespaces extends AdminResource { @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); - } + return super.getPropertyNamespaces(property); } @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") + @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("cluster") String cluster, + 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(property, cluster, namespace); + getNamespacePolicies(namespaceName); try { - return pulsar().getNamespaceService().getListOfDestinations(property, cluster, namespace); + return pulsar().getNamespaceService().getListOfDestinations(namespaceName); } catch (Exception e) { - log.error("Failed to get topics list for namespace {}/{}/{}", property, cluster, namespace, e); + log.error("Failed to get topics list for namespace {}", namespaceName, e); throw new RestException(e); } } @GET - @Path("/{property}/{cluster}/{namespace}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace) { + public Policies getPolicies(@PathParam("property") String property, @PathParam("namespace") String namespace) { validateAdminAccessOnProperty(property); - - return getNamespacePolicies(property, cluster, namespace); + validateNamespaceName(property, namespace); + return getNamespacePolicies(namespaceName); } @PUT - @Path("/{property}/{cluster}/{namespace}") - @ApiOperation(value = "Creates a new empty namespace with no policies attached.") + @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 or namespace doesn't exist"), + @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("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); - } + public void createNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace, + Policies policies) { + validateNamespaceName(property, namespace); - 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); - } + 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}/{cluster}/{namespace}") - @ApiOperation(value = "Delete a namespace and all the destinations under it.") + @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, + public void deleteNamespaceLegacy(@PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + validateNamespaceName(property, cluster, namespace); + internalDeleteNamespace(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}/{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); } @DELETE @Path("/{property}/{cluster}/{namespace}/{bundle}") - @ApiOperation(value = "Delete a namespace bundle and all the destinations under it.") + @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, + public void deleteNamespaceBundleLegacy(@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); - } + validateNamespaceName(property, cluster, namespace); + internalDeleteNamespaceBundle(bundleRange, authoritative); } @GET - @Path("/{property}/{cluster}/{namespace}/permissions") + @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"), @@ -436,49 +177,22 @@ public void deleteNamespaceBundle(@PathParam("property") String property, @PathP public Map> getPermissions(@PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); return policies.auth_policies.namespace_auth; } @POST - @Path("/{property}/{cluster}/{namespace}/permissions/{role}") + @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("cluster") String cluster, + public void grantPermissionOnNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace, @PathParam("role") String role, Set actions) { - 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.put(role, actions); - - // 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 granted access for role {}: {} - namespace {}/{}/{}", clientAppId(), role, - actions, property, cluster, namespace); - } catch (KeeperException.NoNodeException e) { - 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"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to set permissions for namespace {}/{}/{}: concurrent modification", clientAppId(), - property, cluster, namespace); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to get permissions for namespace {}/{}/{}", clientAppId(), property, cluster, - namespace, e); - throw new RestException(e); - } + validateNamespaceName(property, namespace); + internalGrantPermissionOnNamespace(role, actions); } @DELETE @@ -487,739 +201,238 @@ public void grantPermissionOnNamespace(@PathParam("property") String property, @ @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); - } + @PathParam("namespace") String namespace, @PathParam("role") String role) { + validateNamespaceName(property, namespace); + internalRevokePermissionsOnNamespace(role); } @GET - @Path("/{property}/{cluster}/{namespace}/replication") + @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("cluster") String cluster, @PathParam("namespace") String namespace) { + @PathParam("namespace") String namespace) { validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); - 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; + return internalGetNamespaceReplicationClusters(); } @POST - @Path("/{property}/{cluster}/{namespace}/replication") + @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("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); - } + @PathParam("namespace") String namespace, List clusterIds) { + validateNamespaceName(property, namespace); + internalSetNamespaceReplicationClusters(clusterIds); } @GET - @Path("/{property}/{cluster}/{namespace}/messageTTL") + @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("cluster") String cluster, + public int getNamespaceMessageTTL(@PathParam("property") String property, @PathParam("namespace") String namespace) { validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); return policies.message_ttl_in_seconds; } @POST - @Path("/{property}/{cluster}/{namespace}/messageTTL") + @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("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); - } + public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("namespace") String namespace, + int messageTTL) { + validateNamespaceName(property, namespace); + internalSetNamespaceMessageTTL(messageTTL); } @POST - @Path("/{property}/{cluster}/{namespace}/deduplication") + @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("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); - } + public void modifyDeduplication(@PathParam("property") String property, @PathParam("namespace") String namespace, + boolean enableDeduplication) { + validateNamespaceName(property, namespace); + internalModifyDeduplication(enableDeduplication); } @GET - @Path("/{property}/{cluster}/{namespace}/bundles") + @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("cluster") String cluster, + public BundlesData getBundlesData(@PathParam("property") String property, @PathParam("namespace") String namespace) { validateAdminAccessOnProperty(property); validatePoliciesReadOnlyAccess(); + validateNamespaceName(property, namespace); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); 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") + @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 cluster or namespace doesn't exist"), + @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("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); + public void unloadNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + internalUnloadNamespace(); } @PUT - @Path("/{property}/{cluster}/{namespace}/{bundle}/unload") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, + public void unloadNamespaceBundle(@PathParam("property") String property, @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); - } + validateNamespaceName(property, namespace); + internalUnloadNamespaceBundle(bundleRange, authoritative); } @PUT - @Path("/{property}/{cluster}/{namespace}/{bundle}/split") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, + 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) { - 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); - } + validateNamespaceName(property, namespace); + internalSplitNamespaceBundle(bundleRange, authoritative, unload); } @POST - @Path("/{property}/{cluster}/{namespace}/dispatchRate") + @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("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); - } + public void setDispatchRate(@PathParam("property") String property, @PathParam("namespace") String namespace, + DispatchRate dispatchRate) { + validateNamespaceName(property, namespace); + internalSetDispatchRate(dispatchRate); } @GET - @Path("/{property}/{cluster}/{namespace}/dispatchRate") + @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("cluster") String cluster, + public DispatchRate getDispatchRate(@PathParam("property") String property, @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()); - } + validateNamespaceName(property, namespace); + return internalGetDispatchRate(); } @GET - @Path("/{property}/{cluster}/{namespace}/backlogQuotaMap") + @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("cluster") String cluster, @PathParam("namespace") String namespace) { + @PathParam("namespace") String namespace) { validateAdminAccessOnProperty(property); + validateNamespaceName(property, namespace); - Policies policies = getNamespacePolicies(property, cluster, namespace); + Policies policies = getNamespacePolicies(namespaceName); return policies.backlog_quota_map; } @POST - @Path("/{property}/{cluster}/{namespace}/backlogQuota") + @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("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); - } + 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}/{cluster}/{namespace}/backlogQuota") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, + public void removeBacklogQuota(@PathParam("property") String property, @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); - } + validateNamespaceName(property, namespace); + internalRemoveBacklogQuota(backlogQuotaType); } @GET - @Path("/{property}/{cluster}/{namespace}/retention") + @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("cluster") String cluster, + public RetentionPolicies getRetention(@PathParam("property") String property, @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; - } + validateNamespaceName(property, namespace); + return internalGetRetention(); } @POST - @Path("/{property}/{cluster}/{namespace}/retention") + @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("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; + public void setRetention(@PathParam("property") String property, @PathParam("namespace") String namespace, + RetentionPolicies retention) { + validateNamespaceName(property, namespace); + internalSetRetention(retention); } @POST - @Path("/{property}/{cluster}/{namespace}/persistence") + @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("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()); - } + public void setPersistence(@PathParam("property") String property, @PathParam("namespace") String namespace, + PersistencePolicies persistence) { + validateNamespaceName(property, namespace); + internalSetPersistence(persistence); } @GET - @Path("/{property}/{cluster}/{namespace}/persistence") + @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("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; - } + @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetPersistence(); } @POST @@ -1230,397 +443,94 @@ public PersistencePolicies getPersistence(@PathParam("property") String property 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()); + validateNamespaceName(property, namespace); + internalClearNamespaceBacklog(authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange, + @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); + validateNamespaceName(property, namespace); + internalClearNamespaceBundleBacklog(bundleRange, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("subscription") String subscription, + @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()); + validateNamespaceName(property, namespace); + internalClearNamespaceBacklogForSubscription(subscription, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("subscription") String subscription, @PathParam("bundle") String bundleRange, + @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); + validateNamespaceName(property, namespace); + internalClearNamespaceBundleBacklogForSubscription(subscription, bundleRange, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}") + @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) { - 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()); + validateNamespaceName(property, namespace); + internalUnsubscribeNamespace(subscription, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}") - @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace bundle.") + @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("cluster") String cluster, + 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) { - 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); + validateNamespaceName(property, namespace); + internalUnsubscribeNamespaceBundle(subscription, bundleRange, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode") + @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("cluster") String cluster, + public void setSubscriptionAuthMode(@PathParam("property") String property, @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()); - } + validateNamespaceName(property, namespace); + internalSetSubscriptionAuthMode(subscriptionAuthMode); } - /** - * 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") + @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"), + @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, + @ApiResponse(code = 409, message = "Concurrent modification"), }) + public void modifyEncryptionRequired(@PathParam("property") String property, @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); - } + validateNamespaceName(property, 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/NamespacesLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NamespacesLegacy.java new file mode 100644 index 0000000000000..1ad68f01c3a30 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NamespacesLegacy.java @@ -0,0 +1,562 @@ +/** + * 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 org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; + +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 javax.ws.rs.core.Response.Status; + +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.apache.zookeeper.KeeperException; +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 NamespacesLegacy extends NamespacesBase { + + @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}/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, 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, 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, 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(NamespacesLegacy.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/ResourceQuotas.java index 3eed02328def6..44bfdd65efc53 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/ResourceQuotas.java @@ -26,16 +26,9 @@ 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.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.broker.admin.impl.ResourceQuotasBase; 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; @@ -46,135 +39,52 @@ @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) @Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas") -public class ResourceQuotas extends AdminResource { - - private static final Logger log = LoggerFactory.getLogger(ResourceQuotas.class); +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 { - validateSuperUserAccess(); - try { - return pulsar().getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota(); - } catch (Exception e) { - log.error("[{}] Failed to get default resource quota", clientAppId()); - throw new RestException(e); - } - + 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 { - validateSuperUserAccess(); - validatePoliciesReadOnlyAccess(); - try { - pulsar().getLocalZkCacheService().getResourceQuotaCache().setDefaultQuota(quota); - } catch (Exception e) { - log.error("[{}] Failed to get default resource quota", clientAppId()); - throw new RestException(e); - } + super.setDefaultResourceQuota(quota); } @GET - @Path("/{property}/{cluster}/{namespace}/{bundle}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange) { - validateSuperUserAccess(); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } - - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange); - - try { - return pulsar().getLocalZkCacheService().getResourceQuotaCache().getQuota(nsBundle); - } catch (Exception e) { - log.error("[{}] Failed to get resource quota for namespace bundle {}", clientAppId(), nsBundle.toString()); - throw new RestException(e); - } + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) { + validateNamespaceName(property, namespace); + return internalGetNamespaceBundleResourceQuota(bundleRange); } @POST - @Path("/{property}/{cluster}/{namespace}/{bundle}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange, ResourceQuota quota) { - validateSuperUserAccess(); - validatePoliciesReadOnlyAccess(); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } - - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange); - - try { - pulsar().getLocalZkCacheService().getResourceQuotaCache().setQuota(nsBundle, quota); - log.info("[{}] Successfully set resource quota for namespace bundle {}", clientAppId(), - nsBundle.toString()); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to set resource quota for namespace bundle {}: concurrent modification", - clientAppId(), nsBundle.toString()); - throw new RestException(Status.CONFLICT, "Cuncurrent modification on namespace bundle quota"); - } catch (Exception e) { - log.error("[{}] Failed to set resource quota for namespace bundle {}", clientAppId(), nsBundle.toString()); - throw new RestException(e); - } - + @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, ResourceQuota quota) { + validateNamespaceName(property, namespace); + internalSetNamespaceBundleResourceQuota(bundleRange, quota); } @DELETE - @Path("/{property}/{cluster}/{namespace}/{bundle}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("bundle") String bundleRange) { - validateSuperUserAccess(); - validatePoliciesReadOnlyAccess(); - - Policies policies = getNamespacePolicies(property, cluster, namespace); - - if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateClusterOwnership(cluster); - validateClusterForProperty(property, cluster); - } - - NamespaceName fqnn = NamespaceName.get(property, cluster, namespace); - NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange); - - try { - pulsar().getLocalZkCacheService().getResourceQuotaCache().unsetQuota(nsBundle); - log.info("[{}] Successfully unset resource quota for namespace bundle {}", clientAppId(), - nsBundle.toString()); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to unset resource quota for namespace bundle {}: concurrent modification", - clientAppId(), nsBundle.toString()); - throw new RestException(Status.CONFLICT, "Cuncurrent modification on namespace bundle quota"); - } catch (Exception e) { - log.error("[{}] Failed to unset resource quota for namespace bundle {}", clientAppId(), - nsBundle.toString()); - throw new RestException(e); - } + @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/admin/ResourceQuotasLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotasLegacy.java new file mode 100644 index 0000000000000..ea62abcc3e89f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotasLegacy.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; + +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 ResourceQuotasLegacy 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/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java new file mode 100644 index 0000000000000..26a55a569c77d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -0,0 +1,1259 @@ +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 javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +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.client.util.FutureUtil; +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.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; + +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; + +public abstract class NamespacesBase extends AdminResource { + + 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"); + } + } + + 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"); + } + } + + public List getPropertyNamespaces(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()); + 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.put(role, actions); + + // 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 granted access for role {}: {} - namespace {}", clientAppId(), role, actions, + namespaceName); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to set 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 set permissions for namespace {}: concurrent modification", clientAppId(), + namespaceName); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } catch (Exception e) { + 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.getProperty()); + } + + 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); + } + + private static final long MAX_BUNDLES = ((long) 1) << 32; + + 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/impl/ResourceQuotasBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java new file mode 100644 index 0000000000000..9c5a54f8ce592 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java @@ -0,0 +1,135 @@ +/** + * 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 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.policies.data.Policies; +import org.apache.pulsar.common.policies.data.ResourceQuota; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class ResourceQuotasBase extends NamespacesBase { + + public ResourceQuota getDefaultResourceQuota() throws Exception { + validateSuperUserAccess(); + try { + return pulsar().getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota(); + } catch (Exception e) { + log.error("[{}] Failed to get default resource quota", clientAppId()); + throw new RestException(e); + } + + } + + public void setDefaultResourceQuota(ResourceQuota quota) throws Exception { + validateSuperUserAccess(); + validatePoliciesReadOnlyAccess(); + try { + pulsar().getLocalZkCacheService().getResourceQuotaCache().setDefaultQuota(quota); + } catch (Exception e) { + log.error("[{}] Failed to get default resource quota", clientAppId()); + throw new RestException(e); + } + } + + @SuppressWarnings("deprecation") + protected ResourceQuota internalGetNamespaceBundleResourceQuota(String bundleRange) { + validateSuperUserAccess(); + + Policies policies = getNamespacePolicies(namespaceName); + + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange); + + try { + return pulsar().getLocalZkCacheService().getResourceQuotaCache().getQuota(nsBundle); + } catch (Exception e) { + log.error("[{}] Failed to get resource quota for namespace bundle {}", clientAppId(), nsBundle.toString()); + throw new RestException(e); + } + } + + @SuppressWarnings("deprecation") + protected void internalSetNamespaceBundleResourceQuota(String bundleRange, ResourceQuota quota) { + validateSuperUserAccess(); + validatePoliciesReadOnlyAccess(); + + Policies policies = getNamespacePolicies(namespaceName); + + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange); + + try { + pulsar().getLocalZkCacheService().getResourceQuotaCache().setQuota(nsBundle, quota); + log.info("[{}] Successfully set resource quota for namespace bundle {}", clientAppId(), + nsBundle.toString()); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to set resource quota for namespace bundle {}: concurrent modification", + clientAppId(), nsBundle.toString()); + throw new RestException(Status.CONFLICT, "Cuncurrent modification on namespace bundle quota"); + } catch (Exception e) { + log.error("[{}] Failed to set resource quota for namespace bundle {}", clientAppId(), nsBundle.toString()); + throw new RestException(e); + } + + } + + @SuppressWarnings("deprecation") + protected void internalRemoveNamespaceBundleResourceQuota(String bundleRange) { + validateSuperUserAccess(); + validatePoliciesReadOnlyAccess(); + + Policies policies = getNamespacePolicies(namespaceName); + + if (!namespaceName.isGlobal()) { + validateClusterOwnership(namespaceName.getCluster()); + validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster()); + } + + NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange); + + try { + pulsar().getLocalZkCacheService().getResourceQuotaCache().unsetQuota(nsBundle); + log.info("[{}] Successfully unset resource quota for namespace bundle {}", clientAppId(), + nsBundle.toString()); + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to unset resource quota for namespace bundle {}: concurrent modification", + clientAppId(), nsBundle.toString()); + throw new RestException(Status.CONFLICT, "Cuncurrent modification on namespace bundle quota"); + } catch (Exception e) { + log.error("[{}] Failed to unset resource quota for namespace bundle {}", clientAppId(), + nsBundle.toString()); + 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/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 660276656ce56..55406dc1bd4fd 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 @@ -473,7 +473,7 @@ private Optional getLeastLoadedFromLoadManager(ServiceUnitId serviceUnit return Optional.empty(); } } - + public void unloadNamespaceBundle(NamespaceBundle bundle) throws Exception { unloadNamespaceBundle(bundle, 5, TimeUnit.MINUTES); } @@ -607,7 +607,7 @@ public CompletableFuture splitAndOwnBundle(NamespaceBundle bundle, final b String msg = format("bundle %s not found under namespace", bundle.toString()); unloadFuture.completeExceptionally(new ServiceUnitNotReadyException(msg)); } - + return unloadFuture.thenApply(res -> { if (!unload) { return null; @@ -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/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java index 0245b67210ed4..eac1cabd01d21 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 @@ -50,9 +50,9 @@ import org.apache.bookkeeper.util.ZkUtils; 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; @@ -83,14 +83,13 @@ public class AdminTest extends MockedPulsarServiceBaseTest { private Clusters clusters; private Properties properties; - private Namespaces namespaces; + private NamespacesLegacy namespaces; private PersistentTopics persistentTopics; private Brokers brokers; - private ResourceQuotas resourceQuotas; + private ResourceQuotasLegacy resourceQuotas; private BrokerStats brokerStats; private Field uriField; - private UriInfo uriInfo; public AdminTest() { super(); @@ -121,7 +120,7 @@ public void setup() throws Exception { doReturn("test").when(properties).clientAppId(); doNothing().when(properties).validateSuperUserAccess(); - namespaces = spy(new Namespaces()); + namespaces = spy(new NamespacesLegacy()); namespaces.setServletContext(new MockServletContext()); namespaces.setPulsar(pulsar); doReturn(mockZookKeeper).when(namespaces).globalZk(); @@ -145,7 +144,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()); @@ -161,7 +159,7 @@ public void setup() throws Exception { doNothing().when(persistentTopics).validateAdminAccessOnProperty("other-property"); doNothing().when(persistentTopics).validateAdminAccessOnProperty("prop-xyz"); - resourceQuotas = spy(new ResourceQuotas()); + resourceQuotas = spy(new ResourceQuotasLegacy()); resourceQuotas.setServletContext(new MockServletContext()); resourceQuotas.setPulsar(pulsar); doReturn(mockZookKeeper).when(resourceQuotas).globalZk(); @@ -510,7 +508,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 { @@ -569,8 +567,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 @@ -585,13 +582,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); @@ -600,8 +596,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) -> { @@ -634,5 +630,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 37ec7ebc8f170..b51680f0c48e1 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 @@ -83,7 +83,7 @@ @Test public class NamespacesTest extends MockedPulsarServiceBaseTest { - private Namespaces namespaces; + private NamespacesLegacy namespaces; private List testLocalNamespaces; private List testGlobalNamespaces; @@ -121,7 +121,7 @@ public void initNamespace() throws Exception { public void setup() throws Exception { super.internalSetup(); - namespaces = spy(new Namespaces()); + namespaces = spy(new NamespacesLegacy()); namespaces.setServletContext(new MockServletContext()); namespaces.setPulsar(pulsar); doReturn(mockZookKeeper).when(namespaces).globalZk(); 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 7e24e369d5de8..265feea3d0367 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 @@ -97,7 +97,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"); @@ -107,29 +109,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 (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); + } /** @@ -234,9 +251,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()); + } } /** @@ -249,11 +273,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 || "global".equals(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 ea1d1610caf71..b782d9526f42e 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; 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 34b0078214974..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"); @@ -226,32 +219,25 @@ public void testDecodeEncode() throws Exception { @Test public void testTopicNameWithoutCluster() throws Exception { - assertEquals(DestinationName.get("persistent://property/namespace/destination").getNamespace(), - "property/namespace"); + DestinationName dn = DestinationName.get("persistent://property/namespace/destination"); - assertEquals(DestinationName.get("persistent://property/namespace/destination"), - DestinationName.get("persistent", "property", "namespace", "destination")); + assertEquals(dn.getNamespace(), "property/namespace"); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").hashCode(), - DestinationName.get("persistent", "property", "cluster", "namespace", "destination").hashCode()); + assertEquals(dn, DestinationName.get("persistent", "property", "namespace", "destination")); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").toString(), - "persistent://property/cluster/namespace/destination"); + assertEquals(dn.hashCode(), + DestinationName.get("persistent", "property", "namespace", "destination").hashCode()); - assertFalse(DestinationName.get("persistent://property/cluster/namespace/destination") - .equals("persistent://property/cluster/namespace/destination")); + 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(DestinationName.get("persistent://property/cluster/namespace/destination").getDomain(), - DestinationDomain.persistent); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getProperty(), - "property"); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getCluster(), - "cluster"); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getNamespacePortion(), - "namespace"); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getNamespace(), - "property/cluster/namespace"); - assertEquals(DestinationName.get("persistent://property/cluster/namespace/destination").getLocalName(), - "destination"); + assertEquals(dn.getEncodedLocalName(), "destination"); + assertEquals(dn.getPartitionedTopicName(), "persistent://property/namespace/destination"); + assertEquals(dn.getPersistenceNamingEncoding(), "property/namespace/persistent/destination"); } } From 5b32c3a8966b8b25876eeb5617e3bf7b35096451 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 10 Jan 2018 18:50:14 +0100 Subject: [PATCH 03/10] Refactor topics APIs to accept new name formats --- .../pulsar/broker/admin/AdminResource.java | 75 +- .../broker/admin/NonPersistentTopics.java | 78 +- .../admin/NonPersistentTopicsLegacy.java | 161 ++ .../pulsar/broker/admin/PersistentTopics.java | 1436 ++--------------- .../broker/admin/PersistentTopicsLegacy.java | 408 +++++ .../broker/admin/impl/NamespacesBase.java | 20 - .../admin/impl/PersistentTopicsBase.java | 1292 +++++++++++++++ .../pulsar/broker/admin/AdminApiTest.java | 4 +- .../apache/pulsar/broker/admin/AdminTest.java | 4 +- .../pulsar/broker/admin/NamespacesTest.java | 6 +- .../pulsar/common/naming/DestinationName.java | 7 +- 11 files changed, 2101 insertions(+), 1390 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopicsLegacy.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java 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 5ca8c574f3cea..aba41d48aba6e 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 @@ -49,6 +49,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; @@ -202,6 +203,54 @@ 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"); + } + } + + 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); + } + + 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 * @@ -262,7 +311,7 @@ protected ZooKeeperDataCache clustersCache() { return pulsar().getConfigurationCache().clustersCache(); } - ZooKeeperChildrenCache managedLedgerListCache() { + protected ZooKeeperChildrenCache managedLedgerListCache() { return pulsar().getLocalZkCacheService().managedLedgerListCache(); } @@ -286,32 +335,30 @@ protected ZooKeeperDataCache namespaceIsolationPolic return pulsar().getConfigurationCache().namespaceIsolationPoliciesCache(); } - 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; @@ -328,8 +375,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 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/NonPersistentTopics.java index f4429a84639a3..dbb62a6c72d2c 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/NonPersistentTopics.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.admin; import static com.google.common.base.Preconditions.checkNotNull; -import static org.apache.pulsar.common.util.Codec.decode; import javax.ws.rs.DefaultValue; import javax.ws.rs.Encoded; @@ -36,7 +35,6 @@ 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.naming.NamespaceName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.NonPersistentTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; @@ -56,98 +54,92 @@ @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") + @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("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) { - destination = decode(destination); - return getPartitionedTopicMetadata(property, cluster, namespace, destination, authoritative); + validateDestinationName(property, namespace, encodedTopic); + return getPartitionedTopicMetadata(destinationName, authoritative); } @GET - @Path("{property}/{cluster}/{namespace}/{destination}/stats") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public NonPersistentTopicStats getStats(@PathParam("property") String property, + @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, namespace, encodedTopic); + validateAdminOperationOnDestination(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); + return ((NonPersistentTopic) topic).getStats(); } @GET - @Path("{property}/{cluster}/{namespace}/{destination}/internalStats") + @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("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) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); + validateDestinationName(property, namespace, encodedTopic); + validateAdminOperationOnDestination(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); return topic.getInternalStats(); } @PUT - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, int numPartitions, + public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.getProperty()); + 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, 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") + @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("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, namespace, encodedTopic); + log.info("[{}] Unloading topic {}", clientAppId(), destinationName); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); } - unloadTopic(dn, authoritative); + unloadTopic(destinationName, authoritative); } - + protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) { validateAdminAccessOnProperty(fqdn.getProperty()); validateDestinationOwnership(fqdn, authoritative); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java new file mode 100644 index 0000000000000..28a257e16da32 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java @@ -0,0 +1,161 @@ +/** + * 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.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 NonPersistentTopicsLegacy extends PersistentTopics { + private static final Logger log = LoggerFactory.getLogger(NonPersistentTopicsLegacy.class); + + @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 getPartitionedTopicMetadata(destinationName, 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 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) { + validateDestinationName(property, cluster, namespace, encodedTopic); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); + return ((NonPersistentTopic) topic).getStats(); + } + + @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); + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); + return topic.getInternalStats(); + } + + @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); + 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}/{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); + 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/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java index b7361b0276b9b..dfabee01c370c 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/PersistentTopics.java @@ -18,23 +18,9 @@ */ package org.apache.pulsar.broker.admin; -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.io.IOException; -import java.io.OutputStream; -import java.time.Instant; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import javax.ws.rs.DELETE; import javax.ws.rs.DefaultValue; @@ -46,379 +32,103 @@ 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; -import javax.ws.rs.core.StreamingOutput; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ManagedLedgerInfoCallback; -import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; -import org.apache.bookkeeper.mledger.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerOfflineBacklog; -import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; -import org.apache.bookkeeper.mledger.impl.PositionImpl; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; -import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; -import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionInvalidCursorPosition; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; -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.PersistentSubscription; -import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.web.RestException; -import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; -import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; +import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.util.FutureUtil; -import org.apache.pulsar.common.api.Commands; -import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue; -import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; -import org.apache.pulsar.common.compression.CompressionCodec; -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; 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 org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.util.Codec; -import org.apache.pulsar.common.util.DateFormatter; -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.Maps; -import com.google.common.collect.Sets; -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; -import com.github.zafarkhaja.semver.Version; /** */ @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); - - 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); +public class PersistentTopics extends PersistentTopicsBase { @GET - @Path("/{property}/{cluster}/{namespace}") - @ApiOperation(value = "Get the list of destinations under a namespace.", response = String.class, responseContainer = "List") + @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("cluster") String cluster, - @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - // Validate that namespace exists, throws 404 if it doesn't exist - try { - policiesCache().get(path(POLICIES, property, cluster, namespace)); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get topic list {}/{}/{}: Namespace does not exist", clientAppId(), property, - cluster, namespace); - 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); - throw new RestException(e); - } - - List destinations = Lists.newArrayList(); - - try { - String path = String.format("/managed-ledgers/%s/%s/%s/%s", property, cluster, namespace, domain()); - for (String destination : managedLedgerListCache().get(path)) { - if (domain().equals(DestinationDomain.persistent.toString())) { - destinations.add(DestinationName - .get(domain(), property, cluster, namespace, 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); - throw new RestException(e); - } - - destinations.sort(null); - return destinations; + public List getList(@PathParam("property") String property, @PathParam("namespace") String namespace) { + validateNamespaceName(property, namespace); + return internalGetList(); } @GET - @Path("/{property}/{cluster}/{namespace}/partitioned") + @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("cluster") String cluster, + public List getPartitionedTopicList(@PathParam("property") String property, @PathParam("namespace") String namespace) { - validateAdminAccessOnProperty(property); - - // Validate that namespace exists, throws 404 if it doesn't exist - try { - policiesCache().get(path(POLICIES, property, cluster, namespace)); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get partitioned topic list {}/{}/{}: Namespace does not exist", clientAppId(), property, - cluster, namespace); - 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); - throw new RestException(e); - } - - List partitionedTopics = Lists.newArrayList(); - - try { - String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, 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()); - } 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); - throw new RestException(e); - } - - partitionedTopics.sort(null); - return partitionedTopics; + validateNamespaceName(property, namespace); + return internalGetPartitionedTopicList(); } @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/permissions") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination) { - // This operation should be reading from zookeeper and it should be allowed without having admin privileges - destination = decode(destination); - validateAdminAccessOnProperty(property); - - String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString(); - - try { - Policies policies = policiesCache().get(path(POLICIES, property, cluster, namespace)) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); - - Map> permissions = Maps.newTreeMap(); - AuthPolicies auth = policies.auth_policies; - - // First add namespace level permissions - for (String role : auth.namespace_auth.keySet()) { - permissions.put(role, auth.namespace_auth.get(role)); - } - - // Then add destination level permissions - if (auth.destination_auth.containsKey(destinationUri)) { - for (Map.Entry> entry : auth.destination_auth.get(destinationUri).entrySet()) { - String role = entry.getKey(); - Set destinationPermissions = entry.getValue(); - - if (!permissions.containsKey(role)) { - permissions.put(role, destinationPermissions); - } else { - // Do the union between namespace and destination level - Set union = Sets.union(permissions.get(role), destinationPermissions); - permissions.put(role, union); - } - } - } - - return permissions; - } catch (Exception e) { - log.error("[{}] Failed to get permissions for destination {}", clientAppId(), destinationUri, e); - throw new RestException(e); - } - } - - protected void validateAdminAndClientPermission(DestinationName destination) { - try { - validateAdminAccessOnProperty(destination.getProperty()); - } catch (Exception ve) { - try { - checkAuthorization(pulsar(), destination, clientAppId()); - } 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); - throw new RestException(e); - } - } - } - - protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) { - validateAdminAccessOnProperty(fqdn.getProperty()); - validateDestinationOwnership(fqdn, authoritative); + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic) { + validateDestinationName(property, namespace, encodedTopic); + return internalGetPermissionsOnDestination(); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}") - @ApiOperation(value = "Grant a new permission to a role on a single destination.") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, @PathParam("role") String role, Set actions) { - destination = decode(destination); - // This operation should be reading from zookeeper and it should be allowed without having admin privileges - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString(); - - try { - Stat nodeStat = new Stat(); - byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat); - Policies policies = jsonMapper().readValue(content, Policies.class); - - if (!policies.auth_policies.destination_auth.containsKey(destinationUri)) { - policies.auth_policies.destination_auth.put(destinationUri, new TreeMap>()); - } - - 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), - nodeStat.getVersion()); - - // invalidate the local cache to force update - policiesCache().invalidate(path(POLICIES, property, cluster, namespace)); - - log.info("[{}] Successfully granted access for role {}: {} - destination {}", clientAppId(), role, actions, - destinationUri); - - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to grant permissions on destination {}: Namespace does not exist", clientAppId(), - destinationUri); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (Exception e) { - log.error("[{}] Failed to grant permissions for destination {}", clientAppId(), destinationUri, e); - throw new RestException(e); - } + @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}/{cluster}/{namespace}/{destination}/permissions/{role}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destination, @PathParam("role") String role) { - destination = decode(destination); - // This operation should be reading from zookeeper and it should be allowed without having admin privileges - validateAdminAccessOnProperty(property); - validatePoliciesReadOnlyAccess(); - - String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString(); - Stat nodeStat = new Stat(); - Policies policies; - - try { - byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), 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(), - destinationUri); - throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); - } catch (Exception e) { - log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e); - throw new RestException(e); - } - - if (!policies.auth_policies.destination_auth.containsKey(destinationUri) - || !policies.auth_policies.destination_auth.get(destinationUri).containsKey(role)) { - log.warn("[{}] Failed to revoke permission from role {} on destination: Not set at destination level", - clientAppId(), role, destinationUri); - throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the destination level"); - } - - policies.auth_policies.destination_auth.get(destinationUri).remove(role); - - try { - // Write the new policies to zookeeper - String namespacePath = path(POLICIES, property, cluster, namespace); - globalZk().setData(namespacePath, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); - - // invalidate the local cache to force update - policiesCache().invalidate(namespacePath); - globalZkCache().invalidate(namespacePath); - - log.info("[{}] Successfully revoke access for role {} - destination {}", clientAppId(), role, - destinationUri); - } catch (Exception e) { - log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e); - throw new RestException(e); - } + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @PathParam("role") String role) { + validateDestinationName(property, namespace, encodedTopic); + internalRevokePermissionsOnDestination(role); } @PUT - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, int numPartitions, + public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace, + @PathParam("destination") @Encoded String encodedTopic, int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - validateAdminAccessOnProperty(dn.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()); - 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); - } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), dn); - throw new RestException(Status.CONFLICT, "Partitioned topic already exist"); - } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), dn, e); - throw new RestException(e); - } + validateDestinationName(property, namespace, encodedTopic); + internalCreatePartitionedTopic(numPartitions, authoritative); } /** @@ -437,1134 +147,250 @@ public void createPartitionedTopic(@PathParam("property") String property, @Path * @param numPartitions */ @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @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("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); - 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(); - } catch (Exception e) { - if (e.getCause() instanceof RestException) { - throw (RestException) e.getCause(); - } - log.error("[{}] Failed to update partitioned topic {}", clientAppId(), dn, e.getCause()); - throw new RestException(e.getCause()); - } + 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}/{cluster}/{namespace}/{destination}/partitions") + @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("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) { - destination = decode(destination); - PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(property, cluster, namespace, destination, authoritative); - if (metadata.partitions > 1) { - validateClientVersion(); - } - return metadata; + validateDestinationName(property, namespace, encodedTopic); + return internalGetPartitionedMetadata(authoritative); } @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}/partitions") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public void deletePartitionedTopic(@PathParam("property") String property, @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); - validateAdminAccessOnProperty(dn.getProperty()); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, 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); - pulsar().getAdminClient().persistentTopics().deleteAsync(dn_partition.toString()) - .whenComplete((r, ex) -> { - if (ex != null) { - if (ex instanceof NotFoundException) { - // if the sub-topic is not found, the client might not have called create - // producer or it might have been deleted earlier, so we ignore the 404 error. - // For all other exception, we fail the delete partition method even if a single - // partition is failed to be deleted - if (log.isDebugEnabled()) { - log.debug("[{}] Partition not found: {}", clientAppId(), dn_partition); - } - } else { - future.completeExceptionally(ex); - log.error("[{}] Failed to delete partition {}", clientAppId(), dn_partition, - ex); - return; - } - } else { - log.info("[{}] Deleted partition {}", clientAppId(), dn_partition); - } - if (count.decrementAndGet() == 0) { - future.complete(null); - } - }); - } - future.get(); - } catch (Exception e) { - Throwable t = e.getCause(); - if (t instanceof PreconditionFailedException) { - throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions"); - } else { - throw new RestException(t); - } - } - } - - // 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()); - 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); - } 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); - throw new RestException(e); - } + validateDestinationName(property, namespace, encodedTopic); + internalDeletePartitionedTopic(authoritative); } @PUT - @Path("/{property}/{cluster}/{namespace}/{destination}/unload") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public void unloadTopic(@PathParam("property") String property, @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)); - } - unloadTopic(dn, authoritative); + validateDestinationName(property, namespace, encodedTopic); + internalUnloadTopic(authoritative); } @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public void deleteTopic(@PathParam("property") String property, @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); - if (dn.isGlobal()) { - // Delete is disallowed on global topic - log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), dn); - throw new RestException(Status.FORBIDDEN, "Delete forbidden on global namespace"); - } - try { - topic.delete().get(); - log.info("[{}] Successfully removed topic {}", clientAppId(), dn); - } catch (Exception e) { - Throwable t = e.getCause(); - log.error("[{}] Failed to get delete topic {}", clientAppId(), dn, t); - if (t instanceof TopicBusyException) { - throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions"); - } else { - throw new RestException(t); - } - } + validateDestinationName(property, namespace, encodedTopic); + internalDeleteTopic(authoritative); } @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public List getSubscriptions(@PathParam("property") String property, + @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); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - List subscriptions = Lists.newArrayList(); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, 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())); - } catch (Exception e) { - throw new RestException(e); - } - } else { - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); - - try { - topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName)); - } catch (Exception e) { - log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), dn); - throw new RestException(e); - } - } - - return subscriptions; + validateDestinationName(property, namespace, encodedTopic); + return internalGetSubscriptions(authoritative); } @GET - @Path("{property}/{cluster}/{namespace}/{destination}/stats") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public PersistentTopicStats getStats(@PathParam("property") String property, + @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); - validateAdminAndClientPermission(dn); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - validateDestinationOwnership(dn, authoritative); - Topic topic = getTopicReference(dn); - return topic.getStats(); + validateDestinationName(property, namespace, encodedTopic); + return internalGetStats(authoritative); } @GET - @Path("{property}/{cluster}/{namespace}/{destination}/internalStats") + @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("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) { - 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); - return topic.getInternalStats(); + validateDestinationName(property, namespace, encodedTopic); + return internalGetInternalStats(authoritative); } @GET - @Path("{property}/{cluster}/{namespace}/{destination}/internal-info") + @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("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(); - pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() { - @Override - public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { - asyncResponse.resume((StreamingOutput) output -> { - jsonMapper().writer().writeValue(output, info); - }); - } - - @Override - public void getInfoFailed(ManagedLedgerException exception, Object ctx) { - asyncResponse.resume(exception); - } - }, null); + 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}/{cluster}/{namespace}/{destination}/partitioned-stats") + @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("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) { - destination = decode(destination); - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, 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)); - } - PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata); - try { - for (int i = 0; i < partitionMetadata.partitions; i++) { - PersistentTopicStats partitionStats = pulsar().getAdminClient().persistentTopics() - .getStats(dn.getPartition(i).toString()); - stats.add(partitionStats); - stats.partitions.put(dn.getPartition(i).toString(), partitionStats); - } - } catch (Exception e) { - throw new RestException(e); - } - return stats; + validateDestinationName(property, namespace, encodedTopic); + return internalGetPartitionedStats(authoritative); } @DELETE - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, + 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) { - 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); - if (partitionMetadata.partitions > 0) { - try { - for (int i = 0; i < partitionMetadata.partitions; i++) { - pulsar().getAdminClient().persistentTopics().deleteSubscription(dn.getPartition(i).toString(), - subName); - } - } catch (Exception e) { - if (e instanceof NotFoundException) { - throw new RestException(Status.NOT_FOUND, "Subscription not found"); - } 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); - throw new RestException(e); - } - } - } else { - validateAdminOperationOnDestination(dn, authoritative); - Topic topic = getTopicReference(dn); - try { - Subscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.delete().get(); - log.info("[{}][{}] Deleted subscription {}", clientAppId(), dn, subName); - } catch (Exception e) { - Throwable t = e.getCause(); - if (e instanceof NullPointerException) { - throw new RestException(Status.NOT_FOUND, "Subscription not found"); - } 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); - throw new RestException(t); - } - } - } - + validateDestinationName(property, namespace, encodedTopic); + internalDeleteSubscription(subName, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, + 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) { - 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); - if (partitionMetadata.partitions > 0) { - try { - for (int i = 0; i < partitionMetadata.partitions; i++) { - pulsar().getAdminClient().persistentTopics().skipAllMessages(dn.getPartition(i).toString(), - subName); - } - } catch (Exception e) { - throw new RestException(e); - } - } else { - validateAdminOperationOnDestination(dn, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); - try { - if (subName.startsWith(topic.replicatorPrefix)) { - String remoteCluster = PersistentReplicator.getRemoteCluster(subName); - PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - checkNotNull(repl); - repl.clearBacklog().get(); - } else { - PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.clearBacklog().get(); - } - log.info("[{}] Cleared backlog on {} {}", clientAppId(), dn, 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); - throw new RestException(exception); - } - } - + validateDestinationName(property, namespace, encodedTopic); + internalSkipAllMessages(subName, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("numMessages") int numMessages, + 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) { - 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); - 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); - try { - if (subName.startsWith(topic.replicatorPrefix)) { - String remoteCluster = PersistentReplicator.getRemoteCluster(subName); - PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - checkNotNull(repl); - repl.skipMessages(numMessages).get(); - } else { - PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.skipMessages(numMessages).get(); - } - log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, dn, 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); - throw new RestException(exception); - } + validateDestinationName(property, namespace, encodedTopic); + internalSkipMessages(subName, numMessages, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds, + 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) { - destination = decode(destination); - expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative); + validateDestinationName(property, namespace, encodedTopic); + internalExpireMessages(subName, expireTimeInSeconds, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}") + @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("cluster") String cluster, @PathParam("namespace") String namespace, - @PathParam("destination") @Encoded String destinationName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds, + @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic, + @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); - 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); - } - } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds, dn, - e); - throw new RestException(e); - } - } else { - // validate ownership and redirect if current broker is not owner - validateAdminOperationOnDestination(dn, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); - topic.getReplicators().forEach((subName, replicator) -> { - expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative); - }); - topic.getSubscriptions().forEach((subName, subscriber) -> { - expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative); - }); - } + validateDestinationName(property, namespace, encodedTopic); + internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("timestamp") long timestamp, + 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) { - 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); - - if (partitionMetadata.partitions > 0) { - int numParts = partitionMetadata.partitions; - int numPartException = 0; - Exception partitionException = null; - try { - for (int i = 0; i < numParts; i++) { - pulsar().getAdminClient().persistentTopics().resetCursor(dn.getPartition(i).toString(), subName, - timestamp); - } - } catch (PreconditionFailedException pfe) { - // throw the last exception if all partitions get this error - // any other exception on partition is reported back to user - ++numPartException; - partitionException = pfe; - } catch (Exception e) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, 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); - 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); - } - - } else { - validateAdminOperationOnDestination(dn, authoritative); - log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destination, - subName, timestamp); - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); - if (topic == null) { - throw new RestException(Status.NOT_FOUND, "Topic not found"); - } - try { - PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.resetCursor(timestamp).get(); - log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), dn, subName, timestamp); - } catch (Exception e) { - Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName, - timestamp, e); - if (e instanceof NullPointerException) { - throw new RestException(Status.NOT_FOUND, "Subscription not found"); - } else if (e instanceof NotAllowedException) { - throw new RestException(Status.METHOD_NOT_ALLOWED, e.getMessage()); - } else if (t instanceof SubscriptionInvalidCursorPosition) { - throw new RestException(Status.PRECONDITION_FAILED, - "Unable to find position for timestamp specified -" + t.getMessage()); - } else { - throw new RestException(e); - } - } - } + validateDestinationName(property, namespace, encodedTopic); + internalResetCursor(subName, timestamp, authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, + 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) { - 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, - subName, messageId); - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, authoritative); - - if (partitionMetadata.partitions > 0) { - log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), dn, 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); - if (topic == null) { - throw new RestException(Status.NOT_FOUND, "Topic not found"); - } - try { - 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); - } catch (Exception e) { - Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), dn, - subName, messageId, e); - if (e instanceof NullPointerException) { - throw new RestException(Status.NOT_FOUND, "Subscription not found"); - } else if (t instanceof SubscriptionInvalidCursorPosition) { - throw new RestException(Status.PRECONDITION_FAILED, - "Unable to find position for position specified: " + t.getMessage()); - } else { - throw new RestException(e); - } - } - } + validateDestinationName(property, namespace, encodedTopic); + internalResetCursorOnPosition(subName, authoritative, messageId); } @GET - @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, - @PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition, + 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) { - 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); - 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); - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Skip messages on a non-persistent topic is not allowed"); - } - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); - PersistentReplicator repl = null; - PersistentSubscription sub = null; - Entry entry = null; - if (subName.startsWith(topic.replicatorPrefix)) { - repl = getReplicatorReference(subName, topic); - } else { - sub = (PersistentSubscription) getSubscriptionReference(subName, topic); - } - try { - if (subName.startsWith(topic.replicatorPrefix)) { - entry = repl.peekNthMessage(messagePosition).get(); - } else { - entry = sub.peekNthMessage(messagePosition).get(); - } - checkNotNull(entry); - PositionImpl pos = (PositionImpl) entry.getPosition(); - ByteBuf metadataAndPayload = entry.getDataBuffer(); - - // moves the readerIndex to the payload - MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); - - ResponseBuilder responseBuilder = Response.ok(); - responseBuilder.header("X-Pulsar-Message-ID", pos.toString()); - for (KeyValue keyValue : metadata.getPropertiesList()) { - responseBuilder.header("X-Pulsar-PROPERTY-" + keyValue.getKey(), keyValue.getValue()); - } - if (metadata.hasPublishTime()) { - responseBuilder.header("X-Pulsar-publish-time", DateFormatter.format(metadata.getPublishTime())); - } - if (metadata.hasEventTime()) { - responseBuilder.header("X-Pulsar-event-time", DateFormatter.format(metadata.getEventTime())); - } - if (metadata.hasNumMessagesInBatch()) { - responseBuilder.header("X-Pulsar-num-batch-message", metadata.getNumMessagesInBatch()); - } - - // Decode if needed - CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(metadata.getCompression()); - ByteBuf uncompressedPayload = codec.decode(metadataAndPayload, metadata.getUncompressedSize()); - - // Copy into a heap buffer for output stream compatibility - ByteBuf data = PooledByteBufAllocator.DEFAULT.heapBuffer(uncompressedPayload.readableBytes(), - uncompressedPayload.readableBytes()); - data.writeBytes(uncompressedPayload); - uncompressedPayload.release(); - - StreamingOutput stream = new StreamingOutput() { - - @Override - public void write(OutputStream output) throws IOException, WebApplicationException { - output.write(data.array(), data.arrayOffset(), data.readableBytes()); - data.release(); - } - }; - - return responseBuilder.entity(stream).build(); - } 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); - throw new RestException(exception); - } finally { - if (entry != null) { - entry.release(); - } - } + validateDestinationName(property, namespace, encodedTopic); + return internalPeekNthMessage(subName, messagePosition, authoritative); } @GET - @Path("{property}/{cluster}/{namespace}/{destination}/backlog") + @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("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) { - destination = decode(destination); - validateAdminAccessOnProperty(property); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - // 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)); - } catch (KeeperException.NoNodeException e) { - log.warn("[{}] Failed to get topic backlog {}/{}/{}: Namespace does not exist", clientAppId(), property, - cluster, namespace); - 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); - throw new RestException(e); - } - DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination); - PersistentOfflineTopicStats offlineTopicStats = null; - try { - - offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(dn); - if (offlineTopicStats != null) { - // offline topic stat has a cost - so use cached value until TTL - long elapsedMs = System.currentTimeMillis() - offlineTopicStats.statGeneratedAt.getTime(); - if (TimeUnit.MINUTES.convert(elapsedMs, TimeUnit.MILLISECONDS) < OFFLINE_TOPIC_STAT_TTL_MINS) { - return offlineTopicStats; - } - } - final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(dn).get(); - ManagedLedgerOfflineBacklog offlineTopicBacklog = new ManagedLedgerOfflineBacklog(config.getDigestType(), - config.getPassword(), pulsar().getAdvertisedAddress(), false); - offlineTopicStats = offlineTopicBacklog - .estimateUnloadedTopicBacklog((ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), dn); - pulsar().getBrokerService().cacheOfflineTopicStats(dn, offlineTopicStats); - } catch (Exception exception) { - throw new RestException(exception); - } - return offlineTopicStats; + validateDestinationName(property, namespace, encodedTopic); + return internalGetBacklog(authoritative); } @POST - @Path("/{property}/{cluster}/{namespace}/{destination}/terminate") + @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("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, + public MessageId terminate(@PathParam("property") String property, @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); - if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) { - validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, 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); - try { - return ((PersistentTopic) topic).terminate().get(); - } catch (Exception exception) { - log.error("[{}] Failed to terminated topic {}", clientAppId(), dn, 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)); - } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace, - destination, 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); - } - } 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); - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Expire messages on a non-persistent topic is not allowed"); - } - PersistentTopic topic = (PersistentTopic) getTopicReference(dn); - try { - if (subName.startsWith(topic.replicatorPrefix)) { - String remoteCluster = PersistentReplicator.getRemoteCluster(subName); - PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - checkNotNull(repl); - repl.expireMessages(expireTimeInSeconds); - } else { - PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.expireMessages(expireTimeInSeconds); - } - log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds, dn, - 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); - throw new RestException(exception); - } - } - } - - public static CompletableFuture getPartitionedTopicMetadata(PulsarService pulsar, - String clientAppId, DestinationName dn) { - CompletableFuture metadataFuture = new CompletableFuture<>(); - try { - // (1) authorize client - try { - checkAuthorization(pulsar, dn, clientAppId); - } catch (RestException e) { - try { - validateAdminAccessOnProperty(pulsar, clientAppId, dn.getProperty()); - } catch (RestException authException) { - log.warn("Failed to authorize {} on cluster {}", clientAppId, dn.toString()); - throw new PulsarClientException(String.format("Authorization failed %s on cluster %s with error %s", - clientAppId, dn.toString(), authException.getMessage())); - } - } catch (Exception ex) { - // throw without wrapping to PulsarClientException that considers: unknown error marked as internal - // server error - log.warn("Failed to authorize {} on cluster {} with unexpected exception {}", clientAppId, - dn.toString(), ex.getMessage(), ex); - throw ex; - } - - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), - dn.getNamespacePortion(), "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 - // producer/consumer - checkLocalOrGetPeerReplicationCluster(pulsar, dn.getNamespaceObject()) - .thenCompose(res -> fetchPartitionedTopicMetadataAsync(pulsar, path)).thenAccept(metadata -> { - if (log.isDebugEnabled()) { - log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId, dn, - metadata.partitions); - } - metadataFuture.complete(metadata); - }).exceptionally(ex -> { - metadataFuture.completeExceptionally(ex.getCause()); - return null; - }); - } catch (Exception ex) { - metadataFuture.completeExceptionally(ex); - } - return metadataFuture; - } - - /** - * Get the Topic object reference from the Pulsar broker - */ - 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"); - } - } - - /** - * Get the Subscription object reference from the Topic reference - */ - private Subscription getSubscriptionReference(String subName, PersistentTopic topic) { - try { - Subscription sub = topic.getSubscription(subName); - return checkNotNull(sub); - } catch (Exception e) { - throw new RestException(Status.NOT_FOUND, "Subscription not found"); - } - } - - /** - * Get the Replicator object reference from the Topic reference - */ - private PersistentReplicator getReplicatorReference(String replName, PersistentTopic topic) { - try { - String remoteCluster = PersistentReplicator.getRemoteCluster(replName); - PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - return checkNotNull(repl); - } catch (Exception e) { - throw new RestException(Status.NOT_FOUND, "Replicator not found"); - } - } - - private CompletableFuture updatePartitionedTopic(DestinationName dn, int numPartitions) { - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(), - domain(), dn.getEncodedLocalName()); - - CompletableFuture updatePartition = new CompletableFuture<>(); - createSubscriptions(dn, numPartitions).thenAccept(res -> { - try { - byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); - globalZk().setData(path, data, -1, (rc, path1, ctx, stat) -> { - if (rc == KeeperException.Code.OK.intValue()) { - updatePartition.complete(null); - } else { - updatePartition.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), - "failed to create update partitions")); - } - }, null); - } catch (Exception e) { - updatePartition.completeExceptionally(e); - } - }).exceptionally(ex -> { - updatePartition.completeExceptionally(ex); - return null; - }); - - return updatePartition; - } - - /** - * 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 - */ - private CompletableFuture createSubscriptions(DestinationName dn, int numPartitions) { - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(), - domain(), dn.getEncodedLocalName()); - CompletableFuture result = new CompletableFuture<>(); - fetchPartitionedTopicMetadataAsync(pulsar(), path).thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions <= 1) { - result.completeExceptionally(new RestException(Status.CONFLICT, "Topic is not partitioned topic")); - return; - } - - if (partitionMetadata.partitions >= numPartitions) { - result.completeExceptionally(new RestException(Status.CONFLICT, - "number of partitions must be more than existing " + partitionMetadata.partitions)); - return; - } - - // get list of cursors name of partition-1 - final String ledgerName = dn.getPartition(1).getPersistenceNamingEncoding(); - final Set topics = Sets.newConcurrentHashSet(); - ((ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory()).getMetaStore().getCursors(ledgerName, - new MetaStoreCallback>() { - - @Override - public void operationComplete(List cursors, - org.apache.bookkeeper.mledger.impl.MetaStore.Stat stat) { - List> subscriptionCreationFuture = Lists.newArrayList(); - // create subscriptions for all new partition-topics - cursors.forEach(cursor -> { - String subName = Codec.decode(cursor); - for (int i = partitionMetadata.partitions; i < numPartitions; i++) { - final String topicName = dn.getPartition(i).toString(); - CompletableFuture future = new CompletableFuture<>(); - pulsar().getBrokerService().getTopic(topicName).handle((topic, ex) -> { - // cache topic to close all of them after creating all subscriptions - topics.add(topic); - if (ex != null) { - log.warn("[{}] Failed to create topic {}", clientAppId(), topicName); - future.completeExceptionally(ex); - return null; - } else { - topic.createSubscription(subName).handle((sub, e) -> { - if (e != null) { - log.warn("[{}] Failed to create subsciption {} {}", clientAppId(), - topicName, subName); - future.completeExceptionally(e); - return null; - } else { - log.info("[{}] Successfully created subsciption {} {}", - clientAppId(), topicName, subName); - future.complete(null); - return null; - } - }); - return null; - } - }); - subscriptionCreationFuture.add(future); - } - }); - // wait for all subscriptions to be created - FutureUtil.waitForAll(subscriptionCreationFuture).handle((res, subscriptionException) -> { - // close all topics and then complete result future - FutureUtil.waitForAll( - topics.stream().map(topic -> topic.close()).collect(Collectors.toList())) - .handle((closed, topicCloseException) -> { - if (topicCloseException != null) { - log.warn("Failed to close newly created partitioned topics for {} ", dn, - topicCloseException); - } - if (subscriptionException != null) { - result.completeExceptionally(subscriptionException); - } else { - log.info("[{}] Successfully created new partitions {}", clientAppId(), - dn.toString()); - result.complete(null); - } - return null; - }); - return null; - }); - } - - @Override - public void operationFailed(MetaStoreException ex) { - log.warn("[{}] Failed to get list of cursors of {}", clientAppId(), ledgerName); - result.completeExceptionally(ex); - } - }); - }).exceptionally(ex -> { - log.warn("[{}] Failed to get partition metadata for {}", clientAppId(), dn.toString()); - result.completeExceptionally(ex); - return null; - }); - return result; - } - - protected void unloadTopic(DestinationName destination, boolean authoritative) { - validateSuperUserAccess(); - validateDestinationOwnership(destination, authoritative); - try { - Topic topic = getTopicReference(destination); - topic.close().get(); - log.info("[{}] Successfully unloaded topic {}", clientAppId(), destination); - } catch (NullPointerException e) { - log.error("[{}] topic {} not found", clientAppId(), destination); - throw new RestException(Status.NOT_FOUND, "Topic does not exist"); - } catch (Exception e) { - log.error("[{}] Failed to unload topic {}, {}", clientAppId(), destination, e.getCause().getMessage(), e); - throw new RestException(e.getCause()); - } - } - - // as described at : (PR: #836) CPP-client old client lib should not be allowed to connect on partitioned-topic. - // So, all requests from old-cpp-client (< v1.21) must be rejected. - // Pulsar client-java lib always passes user-agent as X-Java-$version. - // However, cpp-client older than v1.20 (PR #765) never used to pass it. - // So, request without user-agent and Pulsar-CPP-vX (X < 1.21) must be rejected - private void validateClientVersion() { - if (!pulsar().getConfiguration().isClientLibraryVersionCheckEnabled()) { - return; - } - final String userAgent = httpRequest.getHeader("User-Agent"); - if (StringUtils.isBlank(userAgent)) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Client lib is not compatible to access partitioned metadata: version in user-agent is not present"); - } - // Version < 1.20 for cpp-client is not allowed - if (userAgent.contains(DEPRECATED_CLIENT_VERSION_PREFIX)) { - try { - // Version < 1.20 for cpp-client is not allowed - String[] tokens = userAgent.split(DEPRECATED_CLIENT_VERSION_PREFIX); - String[] splits = tokens.length > 1 ? tokens[1].split("-")[0].trim().split("\\.") : null; - if (splits != null && splits.length > 1) { - if (LEAST_SUPPORTED_CLIENT_VERSION_PREFIX.getMajorVersion() > Integer.parseInt(splits[0]) - || LEAST_SUPPORTED_CLIENT_VERSION_PREFIX.getMinorVersion() > Integer.parseInt(splits[1])) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Client lib is not compatible to access partitioned metadata: version " + userAgent - + " is not supported"); - } - } - } catch (RestException re) { - throw re; - } catch (Exception e) { - log.warn("[{}] Failed to parse version {} ", clientAppId(), userAgent); - } - } - return; + validateDestinationName(property, namespace, encodedTopic); + return internalTerminate(authoritative); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopicsLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopicsLegacy.java new file mode 100644 index 0000000000000..b435f4de4cf8d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopicsLegacy.java @@ -0,0 +1,408 @@ +/** + * 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 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 PersistentTopicsLegacy 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 destination + * @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); + } + + @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/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index 26a55a569c77d..e5ce93e8f6d42 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -64,26 +64,6 @@ public abstract class NamespacesBase extends AdminResource { - 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"); - } - } - - 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"); - } - } - public List getPropertyNamespaces(String property) { validateAdminAccessOnProperty(property); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java new file mode 100644 index 0000000000000..0d710d0528ff0 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -0,0 +1,1292 @@ +/** + * 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.checkNotNull; +import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.apache.pulsar.common.util.Codec.decode; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.container.AsyncResponse; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.ResponseBuilder; +import javax.ws.rs.core.Response.Status; +import javax.ws.rs.core.StreamingOutput; + +import org.apache.bookkeeper.mledger.AsyncCallbacks.ManagedLedgerInfoCallback; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; +import org.apache.bookkeeper.mledger.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerOfflineBacklog; +import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; +import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; +import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionInvalidCursorPosition; +import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; +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.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; +import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.util.FutureUtil; +import org.apache.pulsar.common.api.Commands; +import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue; +import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; +import org.apache.pulsar.common.compression.CompressionCodec; +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.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AuthPolicies; +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 org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.DateFormatter; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.github.zafarkhaja.semver.Version; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; + +/** + */ +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); + + protected List internalGetList() { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + // Validate that namespace exists, throws 404 if it doesn't exist + try { + policiesCache().get(path(POLICIES, namespaceName.toString())); + } catch (KeeperException.NoNodeException e) { + 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(), namespaceName, e); + throw new RestException(e); + } + + List destinations = Lists.newArrayList(); + + try { + 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(), 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(), namespaceName, e); + throw new RestException(e); + } + + destinations.sort(null); + return destinations; + } + + protected List internalGetPartitionedTopicList() { + validateAdminAccessOnProperty(namespaceName.getProperty()); + + // Validate that namespace exists, throws 404 if it doesn't exist + try { + policiesCache().get(path(POLICIES, namespaceName.toString())); + } catch (KeeperException.NoNodeException e) { + 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(), namespaceName, e); + throw new RestException(e); + } + + List partitionedTopics = Lists.newArrayList(); + + try { + 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", 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(), + namespaceName.toString(), e); + throw new RestException(e); + } + + partitionedTopics.sort(null); + return partitionedTopics; + } + + protected Map> internalGetPermissionsOnDestination() { + // This operation should be reading from zookeeper and it should be allowed without having admin privileges + validateAdminAccessOnProperty(namespaceName.getProperty()); + + String destinationUri = destinationName.toString(); + + try { + Policies policies = policiesCache().get(path(POLICIES, namespaceName.toString())) + .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist")); + + Map> permissions = Maps.newTreeMap(); + AuthPolicies auth = policies.auth_policies; + + // First add namespace level permissions + for (String role : auth.namespace_auth.keySet()) { + permissions.put(role, auth.namespace_auth.get(role)); + } + + // Then add destination level permissions + if (auth.destination_auth.containsKey(destinationUri)) { + for (Map.Entry> entry : auth.destination_auth.get(destinationUri).entrySet()) { + String role = entry.getKey(); + Set destinationPermissions = entry.getValue(); + + if (!permissions.containsKey(role)) { + permissions.put(role, destinationPermissions); + } else { + // Do the union between namespace and destination level + Set union = Sets.union(permissions.get(role), destinationPermissions); + permissions.put(role, union); + } + } + } + + return permissions; + } catch (Exception e) { + log.error("[{}] Failed to get permissions for destination {}", clientAppId(), destinationUri, e); + throw new RestException(e); + } + } + + protected void validateAdminAndClientPermission() { + try { + validateAdminAccessOnProperty(destinationName.getProperty()); + } catch (Exception ve) { + try { + checkAuthorization(pulsar(), destinationName, clientAppId()); + } 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: {}", + destinationName, clientAppId(), e.getMessage(), e); + throw new RestException(e); + } + } + } + + public void validateAdminOperationOnDestination(boolean authoritative) { + validateAdminAccessOnProperty(destinationName.getProperty()); + validateDestinationOwnership(destinationName, authoritative); + } + + protected void internalGrantPermissionsOnDestination(String role, Set actions) { + // This operation should be reading from zookeeper and it should be allowed without having admin privileges + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + String destinationUri = destinationName.toString(); + + try { + Stat nodeStat = new Stat(); + 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)) { + policies.auth_policies.destination_auth.put(destinationUri, new TreeMap>()); + } + + policies.auth_policies.destination_auth.get(destinationUri).put(role, actions); + + // Write the new policies to zookeeper + globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies), + nodeStat.getVersion()); + + // invalidate the local cache to force update + policiesCache().invalidate(path(POLICIES, namespaceName.toString())); + + log.info("[{}] Successfully granted access for role {}: {} - destination {}", clientAppId(), role, actions, + destinationUri); + + } catch (KeeperException.NoNodeException e) { + log.warn("[{}] Failed to grant permissions on destination {}: Namespace does not exist", clientAppId(), + destinationUri); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (Exception e) { + log.error("[{}] Failed to grant permissions for destination {}", clientAppId(), destinationUri, e); + throw new RestException(e); + } + } + + protected void internalRevokePermissionsOnDestination(String role) { + // This operation should be reading from zookeeper and it should be allowed without having admin privileges + validateAdminAccessOnProperty(namespaceName.getProperty()); + validatePoliciesReadOnlyAccess(); + + String destinationUri = destinationName.toString(); + Stat nodeStat = new Stat(); + Policies policies; + + try { + 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(), + destinationUri); + throw new RestException(Status.NOT_FOUND, "Namespace does not exist"); + } catch (Exception e) { + log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e); + throw new RestException(e); + } + + if (!policies.auth_policies.destination_auth.containsKey(destinationUri) + || !policies.auth_policies.destination_auth.get(destinationUri).containsKey(role)) { + log.warn("[{}] Failed to revoke permission from role {} on destination: Not set at destination level", + clientAppId(), role, destinationUri); + throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the destination level"); + } + + policies.auth_policies.destination_auth.get(destinationUri).remove(role); + + try { + // Write the new policies to zookeeper + String namespacePath = path(POLICIES, namespaceName.toString()); + globalZk().setData(namespacePath, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion()); + + // invalidate the local cache to force update + policiesCache().invalidate(namespacePath); + globalZkCache().invalidate(namespacePath); + + log.info("[{}] Successfully revoke access for role {} - destination {}", clientAppId(), role, + destinationUri); + } catch (Exception e) { + log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e); + throw new RestException(e); + } + } + + 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, 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); + } + } + + /** + * 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 + */ + 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(destinationName, numPartitions).get(); + } catch (Exception e) { + if (e.getCause() instanceof RestException) { + throw (RestException) e.getCause(); + } + log.error("[{}] Failed to update partitioned topic {}", clientAppId(), destinationName, e.getCause()); + throw new RestException(e.getCause()); + } + } + + protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean authoritative) { + PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(destinationName, authoritative); + if (metadata.partitions > 1) { + validateClientVersion(); + } + return metadata; + } + + 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 = destinationName.getPartition(i); + pulsar().getAdminClient().persistentTopics().deleteAsync(dn_partition.toString()) + .whenComplete((r, ex) -> { + if (ex != null) { + if (ex instanceof NotFoundException) { + // if the sub-topic is not found, the client might not have called create + // producer or it might have been deleted earlier, so we ignore the 404 error. + // For all other exception, we fail the delete partition method even if a single + // partition is failed to be deleted + if (log.isDebugEnabled()) { + log.debug("[{}] Partition not found: {}", clientAppId(), dn_partition); + } + } else { + future.completeExceptionally(ex); + log.error("[{}] Failed to delete partition {}", clientAppId(), dn_partition, + ex); + return; + } + } else { + log.info("[{}] Deleted partition {}", clientAppId(), dn_partition); + } + if (count.decrementAndGet() == 0) { + future.complete(null); + } + }); + } + future.get(); + } catch (Exception e) { + Throwable t = e.getCause(); + if (t instanceof PreconditionFailedException) { + throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions"); + } else { + throw new RestException(t); + } + } + } + + // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), + 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(), 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(), destinationName, e); + throw new RestException(e); + } + } + + protected void internalUnloadTopic(boolean authoritative) { + log.info("[{}] Unloading topic {}", clientAppId(), destinationName); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + unloadTopic(destinationName, authoritative); + } + + 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(), destinationName); + throw new RestException(Status.FORBIDDEN, "Delete forbidden on global namespace"); + } + + try { + topic.delete().get(); + log.info("[{}] Successfully removed topic {}", clientAppId(), destinationName); + } catch (Exception e) { + Throwable t = e.getCause(); + 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 { + throw new RestException(t); + } + } + } + + protected List internalGetSubscriptions(boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + + List subscriptions = Lists.newArrayList(); + 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(destinationName.getPartition(0).toString())); + } catch (Exception e) { + throw new RestException(e); + } + } else { + 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(), destinationName); + throw new RestException(e); + } + } + + return subscriptions; + } + + protected PersistentTopicStats internalGetStats(boolean authoritative) { + validateAdminAndClientPermission(); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + validateDestinationOwnership(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); + return topic.getStats(); + } + + protected PersistentTopicInternalStats internalGetInternalStats(boolean authoritative) { + validateAdminAndClientPermission(); + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + validateDestinationOwnership(destinationName, authoritative); + Topic topic = getTopicReference(destinationName); + return topic.getInternalStats(); + } + + 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) { + asyncResponse.resume((StreamingOutput) output -> { + jsonMapper().writer().writeValue(output, info); + }); + } + + @Override + public void getInfoFailed(ManagedLedgerException exception, Object ctx) { + asyncResponse.resume(exception); + } + }, null); + } + + 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 (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata); + try { + for (int i = 0; i < partitionMetadata.partitions; i++) { + PersistentTopicStats partitionStats = pulsar().getAdminClient().persistentTopics() + .getStats(destinationName.getPartition(i).toString()); + stats.add(partitionStats); + stats.partitions.put(destinationName.getPartition(i).toString(), partitionStats); + } + } catch (Exception e) { + throw new RestException(e); + } + return stats; + } + + 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(destinationName.getPartition(i).toString(), subName); + } + } catch (Exception e) { + if (e instanceof NotFoundException) { + throw new RestException(Status.NOT_FOUND, "Subscription not found"); + } else if (e instanceof PreconditionFailedException) { + throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers"); + } else { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e); + throw new RestException(e); + } + } + } else { + validateAdminOperationOnDestination(authoritative); + Topic topic = getTopicReference(destinationName); + try { + Subscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.delete().get(); + log.info("[{}][{}] Deleted subscription {}", clientAppId(), destinationName, subName); + } catch (Exception e) { + Throwable t = e.getCause(); + if (e instanceof NullPointerException) { + throw new RestException(Status.NOT_FOUND, "Subscription not found"); + } else if (t instanceof SubscriptionBusyException) { + throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers"); + } else { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e); + throw new RestException(t); + } + } + } + } + + 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(destinationName.getPartition(i).toString(), subName); + } + } catch (Exception e) { + throw new RestException(e); + } + } else { + validateAdminOperationOnDestination(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); + try { + if (subName.startsWith(topic.replicatorPrefix)) { + String remoteCluster = PersistentReplicator.getRemoteCluster(subName); + PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); + checkNotNull(repl); + repl.clearBacklog().get(); + } else { + PersistentSubscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.clearBacklog().get(); + } + 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(), destinationName, subName, exception); + throw new RestException(exception); + } + } + } + + 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(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); + try { + if (subName.startsWith(topic.replicatorPrefix)) { + String remoteCluster = PersistentReplicator.getRemoteCluster(subName); + PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); + checkNotNull(repl); + repl.skipMessages(numMessages).get(); + } else { + PersistentSubscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.skipMessages(numMessages).get(); + } + 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, destinationName, subName, + exception); + throw new RestException(exception); + } + } + + 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( + destinationName.getPartition(i).toString(), expireTimeInSeconds); + } + } catch (Exception 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(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); + topic.getReplicators().forEach((subName, replicator) -> { + internalExpireMessages(subName, expireTimeInSeconds, authoritative); + }); + topic.getSubscriptions().forEach((subName, subscriber) -> { + internalExpireMessages(subName, expireTimeInSeconds, 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; + int numPartException = 0; + Exception partitionException = null; + try { + for (int i = 0; i < numParts; i++) { + pulsar().getAdminClient().persistentTopics().resetCursor(destinationName.getPartition(i).toString(), + subName, timestamp); + } + } catch (PreconditionFailedException pfe) { + // throw the last exception if all partitions get this error + // any other exception on partition is reported back to user + ++numPartException; + partitionException = pfe; + } catch (Exception 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(), + 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(), + destinationName, subName, timestamp, partitionException); + } + + } else { + validateAdminOperationOnDestination(authoritative); + log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destinationName, + subName, timestamp); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); + if (topic == null) { + throw new RestException(Status.NOT_FOUND, "Topic not found"); + } + try { + PersistentSubscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.resetCursor(timestamp).get(); + 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(), + destinationName, subName, timestamp, e); + if (e instanceof NullPointerException) { + throw new RestException(Status.NOT_FOUND, "Subscription not found"); + } else if (e instanceof NotAllowedException) { + throw new RestException(Status.METHOD_NOT_ALLOWED, e.getMessage()); + } else if (t instanceof SubscriptionInvalidCursorPosition) { + throw new RestException(Status.PRECONDITION_FAILED, + "Unable to find position for timestamp specified -" + t.getMessage()); + } else { + throw new RestException(e); + } + } + } + } + + 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(destinationName, authoritative); + + if (partitionMetadata.partitions > 0) { + 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(authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName); + if (topic == null) { + throw new RestException(Status.NOT_FOUND, "Topic not found"); + } + try { + 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(), + destinationName, subName, messageId); + } catch (Exception e) { + Throwable t = e.getCause(); + 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) { + throw new RestException(Status.PRECONDITION_FAILED, + "Unable to find position for position specified: " + t.getMessage()); + } else { + throw new RestException(e); + } + } + } + } + + 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(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(destinationName); + PersistentReplicator repl = null; + PersistentSubscription sub = null; + Entry entry = null; + if (subName.startsWith(topic.replicatorPrefix)) { + repl = getReplicatorReference(subName, topic); + } else { + sub = (PersistentSubscription) getSubscriptionReference(subName, topic); + } + try { + if (subName.startsWith(topic.replicatorPrefix)) { + entry = repl.peekNthMessage(messagePosition).get(); + } else { + entry = sub.peekNthMessage(messagePosition).get(); + } + checkNotNull(entry); + PositionImpl pos = (PositionImpl) entry.getPosition(); + ByteBuf metadataAndPayload = entry.getDataBuffer(); + + // moves the readerIndex to the payload + MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); + + ResponseBuilder responseBuilder = Response.ok(); + responseBuilder.header("X-Pulsar-Message-ID", pos.toString()); + for (KeyValue keyValue : metadata.getPropertiesList()) { + responseBuilder.header("X-Pulsar-PROPERTY-" + keyValue.getKey(), keyValue.getValue()); + } + if (metadata.hasPublishTime()) { + responseBuilder.header("X-Pulsar-publish-time", DateFormatter.format(metadata.getPublishTime())); + } + if (metadata.hasEventTime()) { + responseBuilder.header("X-Pulsar-event-time", DateFormatter.format(metadata.getEventTime())); + } + if (metadata.hasNumMessagesInBatch()) { + responseBuilder.header("X-Pulsar-num-batch-message", metadata.getNumMessagesInBatch()); + } + + // Decode if needed + CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(metadata.getCompression()); + ByteBuf uncompressedPayload = codec.decode(metadataAndPayload, metadata.getUncompressedSize()); + + // Copy into a heap buffer for output stream compatibility + ByteBuf data = PooledByteBufAllocator.DEFAULT.heapBuffer(uncompressedPayload.readableBytes(), + uncompressedPayload.readableBytes()); + data.writeBytes(uncompressedPayload); + uncompressedPayload.release(); + + StreamingOutput stream = new StreamingOutput() { + + @Override + public void write(OutputStream output) throws IOException, WebApplicationException { + output.write(data.array(), data.arrayOffset(), data.readableBytes()); + data.release(); + } + }; + + return responseBuilder.entity(stream).build(); + } 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, + destinationName, subName, exception); + throw new RestException(exception); + } finally { + if (entry != null) { + entry.release(); + } + } + } + + 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, namespaceName.toString())); + } catch (KeeperException.NoNodeException e) { + 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(), namespaceName, e); + throw new RestException(e); + } + + PersistentOfflineTopicStats offlineTopicStats = null; + try { + + 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(); + if (TimeUnit.MINUTES.convert(elapsedMs, TimeUnit.MILLISECONDS) < OFFLINE_TOPIC_STAT_TTL_MINS) { + return offlineTopicStats; + } + } + 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(), destinationName); + pulsar().getBrokerService().cacheOfflineTopicStats(destinationName, offlineTopicStats); + } catch (Exception exception) { + throw new RestException(exception); + } + return offlineTopicStats; + } + + 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(authoritative); + Topic topic = getTopicReference(destinationName); + try { + return ((PersistentTopic) topic).terminate().get(); + } catch (Exception exception) { + log.error("[{}] Failed to terminated topic {}", clientAppId(), destinationName, exception); + throw new RestException(exception); + } + } + + protected void internalExpireMessages(String subName, int expireTimeInSeconds, boolean authoritative) { + if (destinationName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + 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(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(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(destinationName); + try { + if (subName.startsWith(topic.replicatorPrefix)) { + String remoteCluster = PersistentReplicator.getRemoteCluster(subName); + PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); + checkNotNull(repl); + repl.expireMessages(expireTimeInSeconds); + } else { + PersistentSubscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.expireMessages(expireTimeInSeconds); + } + 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, destinationName, subName, exception); + throw new RestException(exception); + } + } + } + + public static CompletableFuture getPartitionedTopicMetadata(PulsarService pulsar, + String clientAppId, DestinationName dn) { + CompletableFuture metadataFuture = new CompletableFuture<>(); + try { + // (1) authorize client + try { + checkAuthorization(pulsar, dn, clientAppId); + } catch (RestException e) { + try { + validateAdminAccessOnProperty(pulsar, clientAppId, dn.getProperty()); + } catch (RestException authException) { + log.warn("Failed to authorize {} on cluster {}", clientAppId, dn.toString()); + throw new PulsarClientException(String.format("Authorization failed %s on cluster %s with error %s", + clientAppId, dn.toString(), authException.getMessage())); + } + } catch (Exception ex) { + // throw without wrapping to PulsarClientException that considers: unknown error marked as internal + // server error + log.warn("Failed to authorize {} on cluster {} with unexpected exception {}", clientAppId, + dn.toString(), ex.getMessage(), ex); + throw ex; + } + + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), + dn.getNamespacePortion(), "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 + // producer/consumer + checkLocalOrGetPeerReplicationCluster(pulsar, dn.getNamespaceObject()) + .thenCompose(res -> fetchPartitionedTopicMetadataAsync(pulsar, path)).thenAccept(metadata -> { + if (log.isDebugEnabled()) { + log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId, dn, + metadata.partitions); + } + metadataFuture.complete(metadata); + }).exceptionally(ex -> { + metadataFuture.completeExceptionally(ex.getCause()); + return null; + }); + } catch (Exception ex) { + metadataFuture.completeExceptionally(ex); + } + return metadataFuture; + } + + /** + * Get the Topic object reference from the Pulsar broker + */ + 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"); + } + } + + /** + * Get the Subscription object reference from the Topic reference + */ + private Subscription getSubscriptionReference(String subName, PersistentTopic topic) { + try { + Subscription sub = topic.getSubscription(subName); + return checkNotNull(sub); + } catch (Exception e) { + throw new RestException(Status.NOT_FOUND, "Subscription not found"); + } + } + + /** + * Get the Replicator object reference from the Topic reference + */ + private PersistentReplicator getReplicatorReference(String replName, PersistentTopic topic) { + try { + String remoteCluster = PersistentReplicator.getRemoteCluster(replName); + PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); + return checkNotNull(repl); + } catch (Exception e) { + throw new RestException(Status.NOT_FOUND, "Replicator not found"); + } + } + + private CompletableFuture updatePartitionedTopic(DestinationName dn, int numPartitions) { + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(), + domain(), dn.getEncodedLocalName()); + + CompletableFuture updatePartition = new CompletableFuture<>(); + createSubscriptions(dn, numPartitions).thenAccept(res -> { + try { + byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); + globalZk().setData(path, data, -1, (rc, path1, ctx, stat) -> { + if (rc == KeeperException.Code.OK.intValue()) { + updatePartition.complete(null); + } else { + updatePartition.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), + "failed to create update partitions")); + } + }, null); + } catch (Exception e) { + updatePartition.completeExceptionally(e); + } + }).exceptionally(ex -> { + updatePartition.completeExceptionally(ex); + return null; + }); + + return updatePartition; + } + + /** + * 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 + */ + private CompletableFuture createSubscriptions(DestinationName dn, int numPartitions) { + String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(), + domain(), dn.getEncodedLocalName()); + CompletableFuture result = new CompletableFuture<>(); + fetchPartitionedTopicMetadataAsync(pulsar(), path).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions <= 1) { + result.completeExceptionally(new RestException(Status.CONFLICT, "Topic is not partitioned topic")); + return; + } + + if (partitionMetadata.partitions >= numPartitions) { + result.completeExceptionally(new RestException(Status.CONFLICT, + "number of partitions must be more than existing " + partitionMetadata.partitions)); + return; + } + + // get list of cursors name of partition-1 + final String ledgerName = dn.getPartition(1).getPersistenceNamingEncoding(); + final Set topics = Sets.newConcurrentHashSet(); + ((ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory()).getMetaStore().getCursors(ledgerName, + new MetaStoreCallback>() { + + @Override + public void operationComplete(List cursors, + org.apache.bookkeeper.mledger.impl.MetaStore.Stat stat) { + List> subscriptionCreationFuture = Lists.newArrayList(); + // create subscriptions for all new partition-topics + cursors.forEach(cursor -> { + String subName = Codec.decode(cursor); + for (int i = partitionMetadata.partitions; i < numPartitions; i++) { + final String topicName = dn.getPartition(i).toString(); + CompletableFuture future = new CompletableFuture<>(); + pulsar().getBrokerService().getTopic(topicName).handle((topic, ex) -> { + // cache topic to close all of them after creating all subscriptions + topics.add(topic); + if (ex != null) { + log.warn("[{}] Failed to create topic {}", clientAppId(), topicName); + future.completeExceptionally(ex); + return null; + } else { + topic.createSubscription(subName).handle((sub, e) -> { + if (e != null) { + log.warn("[{}] Failed to create subsciption {} {}", clientAppId(), + topicName, subName); + future.completeExceptionally(e); + return null; + } else { + log.info("[{}] Successfully created subsciption {} {}", + clientAppId(), topicName, subName); + future.complete(null); + return null; + } + }); + return null; + } + }); + subscriptionCreationFuture.add(future); + } + }); + // wait for all subscriptions to be created + FutureUtil.waitForAll(subscriptionCreationFuture).handle((res, subscriptionException) -> { + // close all topics and then complete result future + FutureUtil.waitForAll( + topics.stream().map(topic -> topic.close()).collect(Collectors.toList())) + .handle((closed, topicCloseException) -> { + if (topicCloseException != null) { + log.warn("Failed to close newly created partitioned topics for {} ", dn, + topicCloseException); + } + if (subscriptionException != null) { + result.completeExceptionally(subscriptionException); + } else { + log.info("[{}] Successfully created new partitions {}", clientAppId(), + dn.toString()); + result.complete(null); + } + return null; + }); + return null; + }); + } + + @Override + public void operationFailed(MetaStoreException ex) { + log.warn("[{}] Failed to get list of cursors of {}", clientAppId(), ledgerName); + result.completeExceptionally(ex); + } + }); + }).exceptionally(ex -> { + log.warn("[{}] Failed to get partition metadata for {}", clientAppId(), dn.toString()); + result.completeExceptionally(ex); + return null; + }); + return result; + } + + protected void unloadTopic(DestinationName destination, boolean authoritative) { + validateSuperUserAccess(); + validateDestinationOwnership(destination, authoritative); + try { + Topic topic = getTopicReference(destination); + topic.close().get(); + log.info("[{}] Successfully unloaded topic {}", clientAppId(), destination); + } catch (NullPointerException e) { + log.error("[{}] topic {} not found", clientAppId(), destination); + throw new RestException(Status.NOT_FOUND, "Topic does not exist"); + } catch (Exception e) { + log.error("[{}] Failed to unload topic {}, {}", clientAppId(), destination, e.getCause().getMessage(), e); + throw new RestException(e.getCause()); + } + } + + // as described at : (PR: #836) CPP-client old client lib should not be allowed to connect on partitioned-topic. + // So, all requests from old-cpp-client (< v1.21) must be rejected. + // Pulsar client-java lib always passes user-agent as X-Java-$version. + // However, cpp-client older than v1.20 (PR #765) never used to pass it. + // So, request without user-agent and Pulsar-CPP-vX (X < 1.21) must be rejected + private void validateClientVersion() { + if (!pulsar().getConfiguration().isClientLibraryVersionCheckEnabled()) { + return; + } + final String userAgent = httpRequest.getHeader("User-Agent"); + if (StringUtils.isBlank(userAgent)) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Client lib is not compatible to access partitioned metadata: version in user-agent is not present"); + } + // Version < 1.20 for cpp-client is not allowed + if (userAgent.contains(DEPRECATED_CLIENT_VERSION_PREFIX)) { + try { + // Version < 1.20 for cpp-client is not allowed + String[] tokens = userAgent.split(DEPRECATED_CLIENT_VERSION_PREFIX); + String[] splits = tokens.length > 1 ? tokens[1].split("-")[0].trim().split("\\.") : null; + if (splits != null && splits.length > 1) { + if (LEAST_SUPPORTED_CLIENT_VERSION_PREFIX.getMajorVersion() > Integer.parseInt(splits[0]) + || LEAST_SUPPORTED_CLIENT_VERSION_PREFIX.getMinorVersion() > Integer.parseInt(splits[1])) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Client lib is not compatible to access partitioned metadata: version " + userAgent + + " is not supported"); + } + } + } catch (RestException re) { + throw re; + } catch (Exception e) { + log.warn("[{}] Failed to parse version {} ", clientAppId(), userAgent); + } + } + return; + } +} 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 795d8071abc37..77250854606c9 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 @@ -1489,7 +1489,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
@@ -1549,7 +1549,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 eac1cabd01d21..170643882d8a8 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
@@ -84,7 +84,7 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
     private Clusters clusters;
     private Properties properties;
     private NamespacesLegacy namespaces;
-    private PersistentTopics persistentTopics;
+    private PersistentTopicsLegacy persistentTopics;
     private Brokers brokers;
     private ResourceQuotasLegacy resourceQuotas;
     private BrokerStats brokerStats;
@@ -145,7 +145,7 @@ public void setup() throws Exception {
         uriField = PulsarWebResource.class.getDeclaredField("uri");
         uriField.setAccessible(true);
 
-        persistentTopics = spy(new PersistentTopics());
+        persistentTopics = spy(new PersistentTopicsLegacy());
         persistentTopics.setServletContext(new MockServletContext());
         persistentTopics.setPulsar(pulsar);
         doReturn(mockZookKeeper).when(persistentTopics).globalZk();
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 b51680f0c48e1..1867827ace7d4 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
@@ -1066,13 +1066,15 @@ public void testValidateDestinationOwnership() throws Exception {
             mockWebUrl(localWebServiceUrl, testNs);
 
             try {
-                topics.validateAdminOperationOnDestination(topicName, false);
+                topics.validateDestinationName(topicName.getProperty(), topicName.getCluster(),
+                        topicName.getNamespace(), topicName.getEncodedLocalName());
+                topics.validateAdminOperationOnDestination(false);
             } catch (RestException e) {
                 fail("validateAdminAccessOnProperty failed");
             }
 
             try {
-                topics.validateAdminOperationOnDestination(DestinationName.get(""), false);
+                topics.validateAdminOperationOnDestination(false);
                 fail("validateAdminAccessOnProperty failed");
             } catch (Exception e) {
                 // OK
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 265feea3d0367..9c15032cc02cd 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,11 @@ 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);

From 8fa94198af4ab1529e22e25fe63cb7b8a78cc91d Mon Sep 17 00:00:00 2001
From: Chris 
Date: Tue, 30 Jan 2018 12:29:19 -0800
Subject: [PATCH 04/10] Separate the broker admin rest endpoints in different
 pacakges.

v1 - legacy endpoints with clusters name
v2 - endpoints without the cluster name

The new admin v2 endpoints are mounted ad /admin/v2.
---
 .../apache/pulsar/broker/PulsarService.java   |   3 +-
 .../pulsar/broker/admin/AdminResource.java    |   6 +-
 .../BrokerStatsBase.java}                     |  13 +-
 .../{Brokers.java => impl/BrokersBase.java}   |  14 +-
 .../{Clusters.java => impl/ClustersBase.java} |  16 +-
 .../broker/admin/impl/NamespacesBase.java     |  31 ++--
 .../PropertiesBase.java}                      |  16 +-
 .../pulsar/broker/admin/v1/BrokerStats.java   |  32 ++++
 .../pulsar/broker/admin/v1/Brokers.java       |  29 ++++
 .../pulsar/broker/admin/v1/Clusters.java      |  32 ++++
 .../Namespaces.java}                          | 161 +++++++++++++++++-
 .../admin/{ => v1}/NonPersistentTopics.java   |  79 +++++----
 .../PersistentTopics.java}                    |   4 +-
 .../pulsar/broker/admin/v1/Properties.java    |  34 ++++
 .../ResourceQuotas.java}                      |   4 +-
 .../pulsar/broker/admin/v2/BrokerStats.java   |  32 ++++
 .../pulsar/broker/admin/v2/Brokers.java       |  29 ++++
 .../pulsar/broker/admin/v2/Clusters.java      |  32 ++++
 .../broker/admin/{ => v2}/Namespaces.java     |  38 +----
 .../NonPersistentTopics.java}                 |  71 ++++----
 .../admin/{ => v2}/PersistentTopics.java      |   2 +-
 .../pulsar/broker/admin/v2/Properties.java    |  34 ++++
 .../broker/admin/{ => v2}/ResourceQuotas.java |   2 +-
 .../loadbalance/impl/LoadManagerShared.java   |   6 +-
 .../pulsar/broker/service/ServerCnx.java      |   2 +-
 .../pulsar/broker/web/PulsarWebResource.java  |   8 +-
 .../utils/PulsarBrokerVersionStringUtils.java |   4 +-
 .../apache/pulsar/broker/admin/AdminTest.java |  19 ++-
 .../pulsar/broker/admin/NamespacesTest.java   |  16 +-
 .../AntiAffinityNamespaceGroupTest.java       |   2 +-
 .../pulsar/admin/cli/CmdPersistentTopics.java |   2 +-
 .../pulsar/common/naming/Constants.java       |  26 +++
 .../pulsar/common/naming/DestinationName.java |   2 +-
 .../pulsar/common/naming/NamespaceName.java   |   2 +-
 34 files changed, 597 insertions(+), 206 deletions(-)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{BrokerStats.java => impl/BrokerStatsBase.java} (96%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{Brokers.java => impl/BrokersBase.java} (95%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{Clusters.java => impl/ClustersBase.java} (98%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{Properties.java => impl/PropertiesBase.java} (95%)
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/BrokerStats.java
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Brokers.java
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Clusters.java
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{NamespacesLegacy.java => v1/Namespaces.java} (78%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{ => v1}/NonPersistentTopics.java (83%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{PersistentTopicsLegacy.java => v1/PersistentTopics.java} (99%)
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{ResourceQuotasLegacy.java => v1/ResourceQuotas.java} (97%)
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Brokers.java
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Clusters.java
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{ => v2}/Namespaces.java (93%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{NonPersistentTopicsLegacy.java => v2/NonPersistentTopics.java} (75%)
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{ => v2}/PersistentTopics.java (99%)
 create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/{ => v2}/ResourceQuotas.java (98%)
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java

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 cab83882adbcd..001f50cabf2dd 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",
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 a39d3d238d79a..a036a966e9423 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
@@ -216,6 +216,7 @@ protected void validateNamespaceName(String property, String namespace) {
         }
     }
 
+    @Deprecated
     protected void validateNamespaceName(String property, String cluster, String namespace) {
         try {
             this.namespaceName = NamespaceName.get(property, cluster, namespace);
@@ -241,6 +242,7 @@ protected void validateDestinationName(String property, String namespace, String
         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 {
@@ -305,7 +307,7 @@ protected ZooKeeperDataCache policiesCache() {
         return pulsar().getConfigurationCache().policiesCache();
     }
 
-    ZooKeeperDataCache localPoliciesCache() {
+    protected ZooKeeperDataCache localPoliciesCache() {
         return pulsar().getLocalZkCacheService().policiesCache();
     }
 
@@ -325,7 +327,7 @@ protected Set clusters() {
         }
     }
 
-    ZooKeeperChildrenCache clustersListCache() {
+    protected ZooKeeperChildrenCache clustersListCache() {
         return pulsar().getConfigurationCache().clustersListCache();
     }
 
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
index e5ce93e8f6d42..35b647a70972a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -1,3 +1,21 @@
+/**
+ * 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;
@@ -15,9 +33,6 @@
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
 
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
@@ -58,12 +73,10 @@
 import com.google.common.collect.Sets;
 import com.google.common.collect.Sets.SetView;
 
-import io.swagger.annotations.ApiOperation;
-import io.swagger.annotations.ApiResponse;
-import io.swagger.annotations.ApiResponses;
-
 public abstract class NamespacesBase extends AdminResource {
 
+    private static final long MAX_BUNDLES = ((long) 1) << 32;
+
     public List getPropertyNamespaces(String property) {
         validateAdminAccessOnProperty(property);
 
@@ -488,7 +501,7 @@ protected void internalUnloadNamespace() {
             validateGlobalNamespaceOwnership(namespaceName);
         } else {
             validateClusterOwnership(namespaceName.getCluster());
-            validateClusterForProperty(namespaceName.getProperty(), namespaceName.getProperty());
+            validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster());
         }
 
         Policies policies = getNamespacePolicies(namespaceName);
@@ -1188,8 +1201,6 @@ protected BundlesData validateBundlesData(BundlesData initialBundles) {
         return new BundlesData(bundles);
     }
 
-    private static final long MAX_BUNDLES = ((long) 1) << 32;
-
     protected BundlesData getBundles(int numBundles) {
         if (numBundles <= 0 || numBundles > MAX_BUNDLES) {
             throw new RestException(Status.BAD_REQUEST,
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/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/NamespacesLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
similarity index 78%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NamespacesLegacy.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
index 1ad68f01c3a30..ada0f0d9c2253 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NamespacesLegacy.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
@@ -16,12 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
+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;
@@ -37,8 +40,10 @@
 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;
@@ -49,6 +54,7 @@
 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;
 
@@ -63,7 +69,16 @@
 @Produces(MediaType.APPLICATION_JSON)
 @Consumes(MediaType.APPLICATION_JSON)
 @Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
-public class NamespacesLegacy extends NamespacesBase {
+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 super.getPropertyNamespaces(property);
+    }
 
     @GET
     @Path("/{property}/{cluster}")
@@ -280,6 +295,140 @@ public void setNamespaceMessageTTL(@PathParam("property") String property, @Path
         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")
@@ -301,7 +450,7 @@ public BundlesData getBundlesData(@PathParam("property") String property, @PathP
             @PathParam("namespace") String namespace) {
         validateAdminAccessOnProperty(property);
         validatePoliciesReadOnlyAccess();
-        validateNamespaceName(property, namespace);
+        validateNamespaceName(property, cluster, namespace);
 
         Policies policies = getNamespacePolicies(namespaceName);
 
@@ -466,7 +615,7 @@ public PersistencePolicies getPersistence(@PathParam("property") String property
     public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster,
             @PathParam("namespace") String namespace,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateNamespaceName(property, namespace);
+        validateNamespaceName(property, cluster, namespace);
         internalClearNamespaceBacklog(authoritative);
     }
 
@@ -479,7 +628,7 @@ 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, namespace);
+        validateNamespaceName(property, cluster, namespace);
         internalClearNamespaceBundleBacklog(bundleRange, authoritative);
     }
 
@@ -558,5 +707,5 @@ public void modifyEncryptionRequired(@PathParam("property") String property, @Pa
         internalModifyEncryptionRequired(encryptionRequired);
     }
 
-    private static final Logger log = LoggerFactory.getLogger(NamespacesLegacy.class);
+    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 83%
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 dcfde963d1a7e..b51133f6de695 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,17 +16,10 @@
  * 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;
 import javax.ws.rs.GET;
@@ -38,13 +31,13 @@
 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.client.util.FutureUtil;
-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;
@@ -56,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")
@@ -72,53 +67,56 @@ 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.")
+    @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("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateDestinationName(property, cluster, namespace, encodedTopic);
         return getPartitionedTopicMetadata(destinationName, authoritative);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/stats")
-    @ApiOperation(value = "Get the stats for the topic.")
+    @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 NonPersistentTopicStats getStats(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        validateAdminOperationOnDestination(destinationName, authoritative);
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateAdminOperationOnDestination(authoritative);
         Topic topic = getTopicReference(destinationName);
         return ((NonPersistentTopic) topic).getStats();
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/internalStats")
-    @ApiOperation(value = "Get the internal stats for the topic.")
+    @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("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        validateAdminOperationOnDestination(destinationName, authoritative);
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateAdminOperationOnDestination(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.")
+    @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("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+    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);
         validateAdminAccessOnProperty(destinationName.getProperty());
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
@@ -141,15 +139,16 @@ public void createPartitionedTopic(@PathParam("property") String property, @Path
     }
 
     @PUT
-    @Path("/{property}/{namespace}/{destination}/unload")
-    @ApiOperation(value = "Unload a topic")
+    @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, namespace, encodedTopic);
+        validateDestinationName(property, cluster, namespace, encodedTopic);
         log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
+
         if (destinationName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
@@ -162,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 {
@@ -208,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/PersistentTopicsLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
similarity index 99%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopicsLegacy.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
index b435f4de4cf8d..a980267daf90a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopicsLegacy.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.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.v1;
 
 import java.util.List;
 import java.util.Map;
@@ -57,7 +57,7 @@
 @Path("/persistent")
 @Produces(MediaType.APPLICATION_JSON)
 @Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic")
-public class PersistentTopicsLegacy extends PersistentTopicsBase {
+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")
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/ResourceQuotasLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java
similarity index 97%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotasLegacy.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java
index ea62abcc3e89f..fb77ee0336a75 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotasLegacy.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.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.v1;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
@@ -39,7 +39,7 @@
 @Produces(MediaType.APPLICATION_JSON)
 @Consumes(MediaType.APPLICATION_JSON)
 @Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas")
-public class ResourceQuotasLegacy extends ResourceQuotasBase {
+public class ResourceQuotas extends ResourceQuotasBase {
 
     @GET
     @Path("/{property}/{cluster}/{namespace}/{bundle}")
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/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
similarity index 93%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
index 652f208e8551b..ffecfc8377540 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.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.v2;
 
 import java.util.List;
 import java.util.Map;
@@ -59,8 +59,6 @@
 @Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
 public class Namespaces extends NamespacesBase {
 
-    public static final String GLOBAL_CLUSTER = "global";
-
     @GET
     @Path("/{property}")
     @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set")
@@ -128,19 +126,6 @@ public void deleteNamespace(@PathParam("property") String property, @PathParam("
         internalDeleteNamespace(authoritative);
     }
 
-    @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 deleteNamespaceLegacy(@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}/{namespace}/bundle/{bundle}")
     @ApiOperation(value = "Delete a namespace bundle and all the topics under it.")
@@ -154,20 +139,6 @@ public void deleteNamespaceBundle(@PathParam("property") String property, @PathP
         internalDeleteNamespaceBundle(bundleRange, 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 deleteNamespaceBundleLegacy(@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}/{namespace}/permissions")
     @ApiOperation(value = "Retrieve the permissions for a namespace.")
@@ -196,7 +167,7 @@ public void grantPermissionOnNamespace(@PathParam("property") String property,
     }
 
     @DELETE
-    @Path("/{property}/{cluster}/{namespace}/permissions/{role}")
+    @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") })
@@ -436,12 +407,11 @@ public PersistencePolicies getPersistence(@PathParam("property") String property
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/clearBacklog")
+    @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("cluster") String cluster,
-            @PathParam("namespace") String namespace,
+    public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("namespace") String namespace,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
         validateNamespaceName(property, namespace);
         internalClearNamespaceBacklog(authoritative);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
similarity index 75%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
index 28a257e16da32..d06d043995f4b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopicsLegacy.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
@@ -16,10 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
+package org.apache.pulsar.broker.admin.v2;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.Encoded;
 import javax.ws.rs.GET;
@@ -31,17 +35,24 @@
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
+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.util.FutureUtil;
 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.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
 import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
+import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.zookeeper.KeeperException;
 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;
@@ -52,60 +63,57 @@
 @Path("/non-persistent")
 @Produces(MediaType.APPLICATION_JSON)
 @Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic")
-public class NonPersistentTopicsLegacy extends PersistentTopics {
-    private static final Logger log = LoggerFactory.getLogger(NonPersistentTopicsLegacy.class);
+public class NonPersistentTopics extends PersistentTopics {
+    private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
 
     @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
+    @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("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateDestinationName(property, namespace, encodedTopic);
         return getPartitionedTopicMetadata(destinationName, authoritative);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/stats")
-    @ApiOperation(hidden = true, value = "Get the stats for the topic.")
+    @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("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        validateAdminOperationOnDestination(authoritative);
+        validateDestinationName(property, namespace, encodedTopic);
+        validateAdminOperationOnDestination(destinationName, authoritative);
         Topic topic = getTopicReference(destinationName);
         return ((NonPersistentTopic) topic).getStats();
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
-    @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
+    @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("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        validateAdminOperationOnDestination(authoritative);
+        validateDestinationName(property, namespace, encodedTopic);
+        validateAdminOperationOnDestination(destinationName, authoritative);
         Topic topic = getTopicReference(destinationName);
         return topic.getInternalStats();
     }
 
     @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.")
+    @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("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);
+    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");
@@ -128,16 +136,15 @@ public void createPartitionedTopic(@PathParam("property") String property, @Path
     }
 
     @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
-    @ApiOperation(hidden = true, value = "Unload a topic")
+    @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("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+    public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateDestinationName(property, namespace, encodedTopic);
         log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
-
         if (destinationName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
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/v2/PersistentTopics.java
similarity index 99%
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/v2/PersistentTopics.java
index dfabee01c370c..b52a5add90e05 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/v2/PersistentTopics.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.v2;
 
 import java.util.List;
 import java.util.Map;
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/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java
similarity index 98%
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/v2/ResourceQuotas.java
index 44bfdd65efc53..b84a14ff0e20f 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/v2/ResourceQuotas.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.v2;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
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 ffdaac64f8e7c..1c43925f42998 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
@@ -239,7 +239,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.
      */
@@ -288,14 +288,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/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
index bd49fc76f6074..23343622e0289 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;
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 40b7d03f2fdca..1fe4fa8df8aec 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,12 +45,8 @@
 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.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;
@@ -308,7 +304,7 @@ protected static boolean isValidCluster(PulsarService pulsarSevice, String clust
         // "global", don't validate the
         // cluster ownership.
         // The validation will be done by checking the namespace configuration
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
+        if (cluster.equals(Constants.GLOBAL_CLUSTER)) {
             return true;
         }
 
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/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
index 8152b0de50562..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,6 +48,13 @@
 
 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.web.PulsarWebResource;
@@ -83,10 +90,10 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
 
     private Clusters clusters;
     private Properties properties;
-    private NamespacesLegacy namespaces;
-    private PersistentTopicsLegacy persistentTopics;
+    private Namespaces namespaces;
+    private PersistentTopics persistentTopics;
     private Brokers brokers;
-    private ResourceQuotasLegacy resourceQuotas;
+    private ResourceQuotas resourceQuotas;
     private BrokerStats brokerStats;
 
     private Field uriField;
@@ -121,7 +128,7 @@ public void setup() throws Exception {
         doReturn("test").when(properties).clientAppId();
         doNothing().when(properties).validateSuperUserAccess();
 
-        namespaces = spy(new NamespacesLegacy());
+        namespaces = spy(new Namespaces());
         namespaces.setServletContext(new MockServletContext());
         namespaces.setPulsar(pulsar);
         doReturn(mockZookKeeper).when(namespaces).globalZk();
@@ -146,7 +153,7 @@ public void setup() throws Exception {
         uriField = PulsarWebResource.class.getDeclaredField("uri");
         uriField.setAccessible(true);
 
-        persistentTopics = spy(new PersistentTopicsLegacy());
+        persistentTopics = spy(new PersistentTopics());
         persistentTopics.setServletContext(new MockServletContext());
         persistentTopics.setPulsar(pulsar);
         doReturn(mockZookKeeper).when(persistentTopics).globalZk();
@@ -160,7 +167,7 @@ public void setup() throws Exception {
         doNothing().when(persistentTopics).validateAdminAccessOnProperty("other-property");
         doNothing().when(persistentTopics).validateAdminAccessOnProperty("prop-xyz");
 
-        resourceQuotas = spy(new ResourceQuotasLegacy());
+        resourceQuotas = spy(new ResourceQuotas());
         resourceQuotas.setServletContext(new MockServletContext());
         resourceQuotas.setPulsar(pulsar);
         doReturn(mockZookKeeper).when(resourceQuotas).globalZk();
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 c6ae9cf0885d1..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;
@@ -83,7 +85,7 @@
 @Test
 public class NamespacesTest extends MockedPulsarServiceBaseTest {
 
-    private NamespacesLegacy namespaces;
+    private Namespaces namespaces;
 
     private List testLocalNamespaces;
     private List testGlobalNamespaces;
@@ -121,7 +123,7 @@ public void initNamespace() throws Exception {
     public void setup() throws Exception {
         super.internalSetup();
 
-        namespaces = spy(new NamespacesLegacy());
+        namespaces = spy(new Namespaces());
         namespaces.setServletContext(new MockServletContext());
         namespaces.setPulsar(pulsar);
         doReturn(mockZookKeeper).when(namespaces).globalZk();
@@ -1064,22 +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.validateDestinationName(topicName.getProperty(), topicName.getCluster(),
-                        topicName.getNamespace(), topicName.getEncodedLocalName());
+                        topicName.getNamespacePortion(), topicName.getEncodedLocalName());
                 topics.validateAdminOperationOnDestination(false);
             } catch (RestException e) {
                 fail("validateAdminAccessOnProperty failed");
             }
 
-            try {
-                topics.validateAdminOperationOnDestination(false);
-                fail("validateAdminAccessOnProperty failed");
-            } catch (Exception e) {
-                // OK
-            }
-
         } catch (RestException e) {
             fail("validateAdminAccessOnProperty failed");
         }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java
index ec0bf1f54e048..5069d3cee8807 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java
@@ -189,7 +189,7 @@ public void testClusterDomain() {
      * It verifies anti-affinity-namespace assignment with failure-domain
      * 
      * 
-     * Domain     Brokers-count
+     * Domain     BrokersBase-count
      * ________  ____________
      * domain-0   broker-0,broker-1
      * domain-1   broker-2,broker-3
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java
index d7f7d51d82a5c..05b467df835af 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java
@@ -501,7 +501,7 @@ void run() throws PulsarAdminException {
                     System.out.println("Message ID: " + msgId.getLedgerId() + ":" + msgId.getEntryId());
                 }
                 if (msg.getProperties().size() > 0) {
-                    System.out.println("Properties:");
+                    System.out.println("PropertiesBase:");
                     print(msg.getProperties());
                 }
                 ByteBuf data = Unpooled.wrappedBuffer(msg.getData());
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 9c15032cc02cd..257badcb23c89 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
@@ -284,7 +284,7 @@ public String getLookupName() {
     }
 
     public boolean isGlobal() {
-        return cluster == null || "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 b782d9526f42e..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
@@ -119,7 +119,7 @@ public String getLocalName() {
     }
 
     public boolean isGlobal() {
-        return cluster == null || "global".equals(cluster);
+        return cluster == null || Constants.GLOBAL_CLUSTER.equalsIgnoreCase(cluster);
     }
 
     public String getPersistentTopicName(String localTopic) {

From 67bcc569e42d1de8fcf6cb57bfd97ff05fc5cbe9 Mon Sep 17 00:00:00 2001
From: Chris 
Date: Tue, 30 Jan 2018 12:37:57 -0800
Subject: [PATCH 05/10] Remove refactor remnants.

---
 .../broker/loadbalance/AntiAffinityNamespaceGroupTest.java      | 2 +-
 .../java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java
index 5069d3cee8807..ec0bf1f54e048 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java
@@ -189,7 +189,7 @@ public void testClusterDomain() {
      * It verifies anti-affinity-namespace assignment with failure-domain
      * 
      * 
-     * Domain     BrokersBase-count
+     * Domain     Brokers-count
      * ________  ____________
      * domain-0   broker-0,broker-1
      * domain-1   broker-2,broker-3
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java
index 05b467df835af..d7f7d51d82a5c 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java
@@ -501,7 +501,7 @@ void run() throws PulsarAdminException {
                     System.out.println("Message ID: " + msgId.getLedgerId() + ":" + msgId.getEntryId());
                 }
                 if (msg.getProperties().size() > 0) {
-                    System.out.println("PropertiesBase:");
+                    System.out.println("Properties:");
                     print(msg.getProperties());
                 }
                 ByteBuf data = Unpooled.wrappedBuffer(msg.getData());

From 6520f0c39520a31014e4a303338c87c7bcc91aba Mon Sep 17 00:00:00 2001
From: Chris 
Date: Wed, 31 Jan 2018 12:15:19 -0800
Subject: [PATCH 06/10] Fix list namespaces to handle v1 and v2 formats.

---
 .../pulsar/broker/admin/AdminResource.java      | 17 ++++++++++++++---
 .../broker/admin/impl/NamespacesBase.java       |  2 +-
 .../pulsar/broker/admin/v1/Namespaces.java      |  2 +-
 .../pulsar/broker/admin/v2/Namespaces.java      |  4 ++--
 4 files changed, 18 insertions(+), 7 deletions(-)

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 a036a966e9423..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
@@ -190,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
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 35b647a70972a..016bb6ecef9df 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -77,7 +77,7 @@ public abstract class NamespacesBase extends AdminResource {
 
     private static final long MAX_BUNDLES = ((long) 1) << 32;
 
-    public List getPropertyNamespaces(String property) {
+    protected List internalGetPropertyNamespaces(String property) {
         validateAdminAccessOnProperty(property);
 
         try {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
index ada0f0d9c2253..3ae1d4bca7495 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
@@ -77,7 +77,7 @@ public class Namespaces extends NamespacesBase {
     @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 super.getPropertyNamespaces(property);
+        return internalGetPropertyNamespaces(property);
     }
 
     @GET
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
index ffecfc8377540..52ea6af3b3eb6 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
@@ -65,7 +65,7 @@ public class Namespaces extends NamespacesBase {
     @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 super.getPropertyNamespaces(property);
+        return internalGetPropertyNamespaces(property);
     }
 
     @GET
@@ -369,7 +369,7 @@ public RetentionPolicies getRetention(@PathParam("property") String property,
     }
 
     @POST
-    @Path("/{property/{namespace}/retention")
+    @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"),

From 5cc32d81c8a6e5023f0b782608274d07938cb94d Mon Sep 17 00:00:00 2001
From: Chris 
Date: Wed, 31 Jan 2018 16:11:26 -0800
Subject: [PATCH 07/10] Add default namespace policies on create if none are
 sent.

---
 .../apache/pulsar/broker/admin/v2/Namespaces.java    | 12 ++++++++++++
 1 file changed, 12 insertions(+)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
index 52ea6af3b3eb6..d00499a4b8e2c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
@@ -111,6 +111,7 @@ public void createNamespace(@PathParam("property") String property, @PathParam("
             Policies policies) {
         validateNamespaceName(property, namespace);
 
+        policies = getDefaultPolicesIfNull(policies);
         internalCreateNamespace(policies);
     }
 
@@ -503,5 +504,16 @@ public void modifyEncryptionRequired(@PathParam("property") String property,
         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);
 }

From 299cd107b09aaab46f4ff6fc38fb01ded1818cd4 Mon Sep 17 00:00:00 2001
From: Chris 
Date: Thu, 8 Feb 2018 09:32:58 -0800
Subject: [PATCH 08/10] Fix internals that assumed a cluster in the path.

---
 .../broker/admin/impl/PersistentTopicsBase.java  |  4 ++--
 .../pulsar/broker/lookup/DestinationLookup.java  |  4 +++-
 .../broker/namespace/ServiceUnitZkUtils.java     | 15 +++++++++++----
 .../pulsar/broker/service/BrokerService.java     |  2 +-
 .../apache/pulsar/broker/service/ServerCnx.java  |  2 +-
 .../pulsar/broker/web/PulsarWebResource.java     | 16 ++++++++--------
 .../pulsar/common/naming/DestinationName.java    |  1 +
 7 files changed, 27 insertions(+), 17 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
index c6b7892fbf8d4..a9640d6021ac2 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
@@ -1093,8 +1093,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
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 a666166c454c0..d21d801d83bdd 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
@@ -51,6 +51,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;
@@ -210,7 +211,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/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 74ca915e3b3ff..c56a027b415b6 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 11ab2a90aa5c2..fa009f134543e 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
@@ -230,7 +230,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 1fe4fa8df8aec..25ee629b3946c 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
@@ -267,12 +267,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))
@@ -300,15 +301,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(Constants.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;
         }
@@ -555,8 +556,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-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 257badcb23c89..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
@@ -181,6 +181,7 @@ public String getProperty() {
         return property;
     }
 
+    @Deprecated
     public String getCluster() {
         return cluster;
     }

From 29fba8bacc1bf186b2600e666437090ead14a0c7 Mon Sep 17 00:00:00 2001
From: Chris 
Date: Thu, 8 Feb 2018 11:22:18 -0800
Subject: [PATCH 09/10] Fix merge compile issue.

---
 .../apache/pulsar/broker/admin/impl/PersistentTopicsBase.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
index 1a141ff536b25..4da6e717af23f 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
@@ -46,13 +46,12 @@
 import org.apache.bookkeeper.mledger.Entry;
 import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
 import org.apache.bookkeeper.mledger.ManagedLedgerException;
-import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException;
 import org.apache.bookkeeper.mledger.ManagedLedgerInfo;
 import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl;
 import org.apache.bookkeeper.mledger.impl.ManagedLedgerOfflineBacklog;
-import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback;
 import org.apache.bookkeeper.mledger.impl.PositionImpl;
 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.service.BrokerServiceException.NotAllowedException;
@@ -88,7 +87,6 @@
 import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
 import org.apache.pulsar.common.policies.data.PersistentTopicStats;
 import org.apache.pulsar.common.policies.data.Policies;
-import org.apache.pulsar.common.util.Codec;
 import org.apache.pulsar.common.util.DateFormatter;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;

From fd04aedb2199955d3cdcd52fdfc11d314b00a31d Mon Sep 17 00:00:00 2001
From: Chris 
Date: Mon, 12 Feb 2018 15:18:38 -0800
Subject: [PATCH 10/10] Fix compile issues from merge with master.

---
 .../apache/pulsar/broker/admin/impl/NamespacesBase.java  | 2 +-
 .../pulsar/broker/admin/v1/NonPersistentTopics.java      | 3 ---
 .../pulsar/broker/admin/v2/NonPersistentTopics.java      | 9 ---------
 .../org/apache/pulsar/broker/web/PulsarWebResource.java  | 1 -
 4 files changed, 1 insertion(+), 14 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 28b51e635da10..0dda3aed06c3d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -49,7 +49,6 @@
 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.client.util.FutureUtil;
 import org.apache.pulsar.common.naming.DestinationName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
@@ -65,6 +64,7 @@
 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;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
index 2bb6889728560..c423f2184a823 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
@@ -36,10 +36,7 @@
 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.util.FutureUtil;
 import org.apache.pulsar.common.naming.Constants;
-import org.apache.pulsar.client.admin.PulsarAdminException;
-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;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
index d06d043995f4b..55bd9be9bedc0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
@@ -20,9 +20,6 @@
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
 
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.Encoded;
@@ -35,23 +32,17 @@
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
-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.util.FutureUtil;
 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.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
 import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
-import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
 
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
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 5877b230bb616..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,7 +45,6 @@
 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;